From 04c2a2d06a450cb08e63bce890c99d347d0b9b4c Mon Sep 17 00:00:00 2001 From: Kenneth Jung Date: Thu, 30 Apr 2020 11:05:08 -0700 Subject: [PATCH 001/151] Reject fractional values outside of (0.0, 1.0) --- .../sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java index f76a4328dfcb..d460c5d3ca40 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java @@ -288,6 +288,11 @@ public BoundedSource splitAtFraction(double fraction) { source.stream.getName(), fraction); + if (fraction <= 0.0 || fraction >= 1.0) { + LOG.info("BigQuery Storage API does not support splitting at fraction {}", fraction); + return null; + } + SplitReadStreamRequest splitRequest = SplitReadStreamRequest.newBuilder() .setOriginalStream(source.stream) From 778e0f9cfa0bb8f456524619ea40b25c24bcb5de Mon Sep 17 00:00:00 2001 From: Esun Kim Date: Fri, 8 May 2020 20:09:40 -0700 Subject: [PATCH 002/151] Upgrades gcsio to 2.1.3 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index eaf79fbdd696..ac53a23daf4a 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -382,7 +382,7 @@ class BeamModulePlugin implements Plugin { def generated_grpc_ga_version = "1.85.1" def google_auth_version = "0.19.0" def google_clients_version = "1.30.9" - def google_cloud_bigdataoss_version = "2.1.2" + def google_cloud_bigdataoss_version = "2.1.3" def google_cloud_core_version = "1.92.2" def google_cloud_spanner_version = "1.49.1" def google_cloud_datacatalog_version = "0.32.1" From 3e70f16d1abd92c27eb6f54330a7427429827074 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 22 Apr 2020 16:11:26 -0700 Subject: [PATCH 003/151] Make GroupByKey a primitive Move old implementation to DirectRunner. Change-Id: I9aaf3c00ed7e9606f10e417f0a504a5fc63ee68b --- sdks/python/apache_beam/pipeline_test.py | 18 +-- .../runners/dataflow/dataflow_runner.py | 16 ++- .../runners/dataflow/dataflow_runner_test.py | 117 +++++++++++------- .../runners/direct/direct_runner.py | 86 +++++++++++++ .../interactive/pipeline_analyzer_test.py | 2 +- .../interactive/pipeline_instrument_test.py | 12 +- .../fn_api_runner/fn_runner_test.py | 4 +- sdks/python/apache_beam/transforms/core.py | 45 +------ 8 files changed, 187 insertions(+), 113 deletions(-) diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index 0011109f5d25..a7e0eac9c860 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -27,7 +27,6 @@ from builtins import object from builtins import range -import mock from nose.plugins.attrib import attr import apache_beam as beam @@ -370,9 +369,7 @@ def raise_exception(exn): # pylint: disable=expression-not-assigned p | Create([ValueError('msg')]) | Map(raise_exception) - @mock.patch( - 'apache_beam.runners.direct.direct_runner._get_transform_overrides') - def test_ptransform_overrides(self, file_system_override_mock): + def test_ptransform_overrides(self): class MyParDoOverride(PTransformOverride): def matches(self, applied_ptransform): return isinstance(applied_ptransform.transform, DoubleParDo) @@ -382,15 +379,12 @@ def get_replacement_transform(self, ptransform): return TripleParDo() raise ValueError('Unsupported type of transform: %r' % ptransform) - def get_overrides(unused_pipeline_options): - return [MyParDoOverride()] + p = Pipeline() + pcoll = p | beam.Create([1, 2, 3]) | 'Multiply' >> DoubleParDo() + assert_that(pcoll, equal_to([3, 6, 9])) - file_system_override_mock.side_effect = get_overrides - - # Specify DirectRunner as it's the one patched above. - with Pipeline(runner='BundleBasedDirectRunner') as p: - pcoll = p | beam.Create([1, 2, 3]) | 'Multiply' >> DoubleParDo() - assert_that(pcoll, equal_to([3, 6, 9])) + p.replace_all([MyParDoOverride()]) + p.run() def test_ptransform_override_type_hints(self): class NoTypeHintOverride(PTransformOverride): diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 07c5f8809fc7..4c02168cee58 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -278,7 +278,7 @@ def group_by_key_input_visitor(): class GroupByKeyInputVisitor(PipelineVisitor): """A visitor that replaces `Any` element type for input `PCollection` of - a `GroupByKey` or `_GroupByKeyOnly` with a `KV` type. + a `GroupByKey` with a `KV` type. TODO(BEAM-115): Once Python SDk is compatible with the new Runner API, we could directly replace the coder instead of mutating the element type. @@ -289,8 +289,8 @@ def enter_composite_transform(self, transform_node): def visit_transform(self, transform_node): # Imported here to avoid circular dependencies. # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.transforms.core import GroupByKey, _GroupByKeyOnly - if isinstance(transform_node.transform, (GroupByKey, _GroupByKeyOnly)): + from apache_beam.transforms.core import GroupByKey + if isinstance(transform_node.transform, GroupByKey): pcoll = transform_node.inputs[0] pcoll.element_type = typehints.coerce_to_kv_type( pcoll.element_type, transform_node.full_label) @@ -840,7 +840,11 @@ def apply_WriteToBigQuery(self, transform, pcoll, options): transform.write_disposition, kms_key=transform.kms_key)) + # TODO(srohde): Remove this after internal usages have been removed. def apply_GroupByKey(self, transform, pcoll, options): + return transform.expand(pcoll) + + def _verify_gbk_coders(self, transform, pcoll): # Infer coder of parent. # # TODO(ccy): make Coder inference and checking less specialized and more @@ -858,11 +862,13 @@ def apply_GroupByKey(self, transform, pcoll, options): coders.registry.verify_deterministic( coder.key_coder(), 'GroupByKey operation "%s"' % transform.label) - return pvalue.PCollection.from_(pcoll) - def run_GroupByKey(self, transform_node, options): input_tag = transform_node.inputs[0].tag input_step = self._cache.get_pvalue(transform_node.inputs[0]) + + # Verify that the GBK's parent has a KV coder. + self._verify_gbk_coders(transform_node.transform, transform_node.inputs[0]) + step = self._add_step( TransformNames.GROUP, transform_node.full_label, transform_node) step.add_property( diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 87f8785e394e..0791b49a983d 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -55,7 +55,6 @@ from apache_beam.transforms import environments from apache_beam.transforms import window from apache_beam.transforms.core import Windowing -from apache_beam.transforms.core import _GroupByKeyOnly from apache_beam.transforms.display import DisplayDataItem from apache_beam.typehints import typehints @@ -315,53 +314,46 @@ def test_remote_runner_display_data(self): def test_no_group_by_key_directly_after_bigquery(self): remote_runner = DataflowRunner() - p = Pipeline( - remote_runner, - options=PipelineOptions([ - '--dataflow_endpoint=ignored', - '--job_name=test-job', - '--project=test-project', - '--staging_location=ignored', - '--temp_location=/dev/null', - '--no_auth' - ])) - rows = p | beam.io.Read(beam.io.BigQuerySource('dataset.faketable')) with self.assertRaises(ValueError, msg=('Coder for the GroupByKey operation' '"GroupByKey" is not a key-value coder: ' 'RowAsDictJsonCoder')): - unused_invalid = rows | beam.GroupByKey() + with beam.Pipeline(runner=remote_runner, + options=PipelineOptions(self.default_properties)) as p: + # pylint: disable=expression-not-assigned + p | beam.io.Read( + beam.io.BigQuerySource('dataset.faketable')) | beam.GroupByKey() def test_group_by_key_input_visitor_with_valid_inputs(self): p = TestPipeline() pcoll1 = PCollection(p) pcoll2 = PCollection(p) pcoll3 = PCollection(p) - for transform in [_GroupByKeyOnly(), beam.GroupByKey()]: - pcoll1.element_type = None - pcoll2.element_type = typehints.Any - pcoll3.element_type = typehints.KV[typehints.Any, typehints.Any] - for pcoll in [pcoll1, pcoll2, pcoll3]: - applied = AppliedPTransform(None, transform, "label", [pcoll]) - applied.outputs[None] = PCollection(None) - DataflowRunner.group_by_key_input_visitor().visit_transform(applied) - self.assertEqual( - pcoll.element_type, typehints.KV[typehints.Any, typehints.Any]) + + pcoll1.element_type = None + pcoll2.element_type = typehints.Any + pcoll3.element_type = typehints.KV[typehints.Any, typehints.Any] + for pcoll in [pcoll1, pcoll2, pcoll3]: + applied = AppliedPTransform(None, beam.GroupByKey(), "label", [pcoll]) + applied.outputs[None] = PCollection(None) + DataflowRunner.group_by_key_input_visitor().visit_transform(applied) + self.assertEqual( + pcoll.element_type, typehints.KV[typehints.Any, typehints.Any]) def test_group_by_key_input_visitor_with_invalid_inputs(self): p = TestPipeline() pcoll1 = PCollection(p) pcoll2 = PCollection(p) - for transform in [_GroupByKeyOnly(), beam.GroupByKey()]: - pcoll1.element_type = str - pcoll2.element_type = typehints.Set - err_msg = ( - r"Input to 'label' must be compatible with KV\[Any, Any\]. " - "Found .*") - for pcoll in [pcoll1, pcoll2]: - with self.assertRaisesRegex(ValueError, err_msg): - DataflowRunner.group_by_key_input_visitor().visit_transform( - AppliedPTransform(None, transform, "label", [pcoll])) + + pcoll1.element_type = str + pcoll2.element_type = typehints.Set + err_msg = ( + r"Input to 'label' must be compatible with KV\[Any, Any\]. " + "Found .*") + for pcoll in [pcoll1, pcoll2]: + with self.assertRaisesRegex(ValueError, err_msg): + DataflowRunner.group_by_key_input_visitor().visit_transform( + AppliedPTransform(None, beam.GroupByKey(), "label", [pcoll])) def test_group_by_key_input_visitor_for_non_gbk_transforms(self): p = TestPipeline() @@ -404,10 +396,6 @@ def test_gbk_then_flatten_input_visitor(self): flat = (none_str_pc, none_int_pc) | beam.Flatten() _ = flat | beam.GroupByKey() - # This may change if type inference changes, but we assert it here - # to make sure the check below is not vacuous. - self.assertNotIsInstance(flat.element_type, typehints.TupleConstraint) - p.visit(DataflowRunner.group_by_key_input_visitor()) p.visit(DataflowRunner.flatten_input_visitor()) @@ -596,6 +584,15 @@ def test_combine_values_translation(self): self.assertIn( u'CombineValues', set(step[u'kind'] for step in job_dict[u'steps'])) + def _find_step(self, job, step_name): + job_dict = json.loads(str(job)) + maybe_step = [ + s for s in job_dict[u'steps'] + if s[u'properties'][u'user_name'] == step_name + ] + self.assertTrue(maybe_step, 'Could not find step {}'.format(step_name)) + return maybe_step[0] + def expect_correct_override(self, job, step_name, step_kind): """Expects that a transform was correctly overriden.""" @@ -615,14 +612,7 @@ def expect_correct_override(self, job, step_name, step_kind): "user_name": step_name + ".out" }] - job_dict = json.loads(str(job)) - maybe_step = [ - s for s in job_dict[u'steps'] - if s[u'properties'][u'user_name'] == step_name - ] - self.assertTrue(maybe_step, 'Could not find step {}'.format(step_name)) - - step = maybe_step[0] + step = self._find_step(job, step_name) self.assertEqual(step[u'kind'], step_kind) # The display data here is forwarded because the replace transform is @@ -663,6 +653,43 @@ def test_read_pubsub_translation(self): self.expect_correct_override( runner.job, u'ReadFromPubSub/Read', u'ParallelRead') + def test_gbk_translation(self): + runner = DataflowRunner() + with beam.Pipeline(runner=runner, + options=PipelineOptions(self.default_properties)) as p: + # pylint: disable=expression-not-assigned + p | beam.Create([(1, 2)]) | beam.GroupByKey() + + expected_output_info = [{ + "encoding": { + "@type": "kind:windowed_value", + "component_encodings": [{ + "@type": "kind:pair", + "component_encodings": [{ + "@type": "kind:varint" + }, + { + "@type": "kind:stream", + "component_encodings": [{ + "@type": "kind:varint" + }], + "is_stream_like": True + }], + "is_pair_like": True + }, { + "@type": "kind:global_window" + }], + "is_wrapper": True + }, + "output_name": "out", + "user_name": "GroupByKey.out" + }] # yapf: disable + + gbk_step = self._find_step(runner.job, u'GroupByKey') + self.assertEqual(gbk_step[u'kind'], u'GroupByKey') + self.assertEqual( + gbk_step[u'properties']['output_info'], expected_output_info) + class CustomMergingWindowFn(window.WindowFn): def assign(self, assign_context): diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 5181ad734ff0..fb80f19b5d3b 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -234,7 +234,89 @@ def get_replacement_transform(self, transform): from apache_beam.runners.direct.test_stream_impl import _ExpandableTestStream return _ExpandableTestStream(transform) + class GroupByKeyPTransformOverride(PTransformOverride): + """A ``PTransformOverride`` for ``GroupByKey``. + + This replaces the Beam implementation as a primitive. + """ + def matches(self, applied_ptransform): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.transforms.core import GroupByKey + if (isinstance(applied_ptransform.transform, GroupByKey) and + not getattr(applied_ptransform.transform, 'override', False)): + self.input_type = applied_ptransform.inputs[0].element_type + return True + return False + + def get_replacement_transform(self, ptransform): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.transforms.core import GroupByKey + + # Subclass from GroupByKey to inherit all the proper methods. + class GroupByKey(GroupByKey): + override = True + + def expand(self, pcoll): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.coders import typecoders + from apache_beam.typehints import trivial_inference + + input_type = pcoll.element_type + if input_type is not None: + # Initialize type-hints used below to enforce type-checking and to + # pass downstream to further PTransforms. + key_type, value_type = trivial_inference.key_value_types(input_type) + # Enforce the input to a GBK has a KV element type. + pcoll.element_type = typehints.typehints.coerce_to_kv_type( + pcoll.element_type) + typecoders.registry.verify_deterministic( + typecoders.registry.get_coder(key_type), + 'GroupByKey operation "%s"' % self.label) + + reify_output_type = typehints.KV[ + key_type, typehints.WindowedValue[value_type]] # type: ignore[misc] + gbk_input_type = ( + typehints. + KV[key_type, + typehints.Iterable[ + typehints.WindowedValue[ # type: ignore[misc] + value_type]]]) + gbk_output_type = typehints.KV[key_type, + typehints.Iterable[value_type]] + + # pylint: disable=bad-continuation + return ( + pcoll + | 'ReifyWindows' >> ( + ParDo(self.ReifyWindows()).with_output_types( + reify_output_type)) + | 'GroupByKey' >> ( + _GroupByKeyOnly().with_input_types( + reify_output_type).with_output_types(gbk_input_type)) + | ( + 'GroupByWindow' >> + _GroupAlsoByWindow(pcoll.windowing).with_input_types( + gbk_input_type).with_output_types(gbk_output_type))) + else: + # The input_type is None, run the default + return ( + pcoll + | 'ReifyWindows' >> ParDo(self.ReifyWindows()) + | 'GroupByKey' >> _GroupByKeyOnly() + | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) + + # Infer the correct type. + return GroupByKey().with_output_types( + ptransform.infer_output_type(self.input_type)) + overrides = [ + # This needs to be the first and the last override. Other overrides depend + # on the GroupByKey implementation to be composed of _GroupByKeyOnly and + # _GroupAlsoByWindow. + GroupByKeyPTransformOverride(), SplittableParDoOverride(), ProcessKeyedElementsViaKeyedWorkItemsOverride(), CombinePerKeyOverride(), @@ -253,6 +335,10 @@ def get_replacement_transform(self, transform): except ImportError: pass + # This also needs to be last because other transforms apply GBKs which need to + # be translated into a DirectRunner-compatible transform. + overrides.append(GroupByKeyPTransformOverride()) + return overrides diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_analyzer_test.py b/sdks/python/apache_beam/runners/interactive/pipeline_analyzer_test.py index fdc9145aef10..5cf0921ac804 100644 --- a/sdks/python/apache_beam/runners/interactive/pipeline_analyzer_test.py +++ b/sdks/python/apache_beam/runners/interactive/pipeline_analyzer_test.py @@ -219,7 +219,7 @@ def test_read_cache_expansion(self): pipeline_proto = to_stable_runner_api(p) pipeline_info = pipeline_analyzer.PipelineInfo(pipeline_proto.components) - pcoll_id = 'ref_PCollection_PCollection_12' # Output PCollection of Square + pcoll_id = 'ref_PCollection_PCollection_10' # Output PCollection of Square cache_label1 = pipeline_info.cache_label(pcoll_id) analyzer = pipeline_analyzer.PipelineAnalyzer( diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py index 8ef6f8c65cf7..9f2684e8e83f 100644 --- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py +++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py @@ -69,7 +69,7 @@ def test_cacheable_key_without_version_map(self): _, ctx = p.to_runner_api(use_fake_coders=True, return_context=True) self.assertEqual( instr.cacheable_key(init_pcoll, instr.pcolls_to_pcoll_id(p, ctx)), - str(id(init_pcoll)) + '_ref_PCollection_PCollection_10') + str(id(init_pcoll)) + '_ref_PCollection_PCollection_8') def test_cacheable_key_with_version_map(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) @@ -94,8 +94,8 @@ def test_cacheable_key_with_version_map(self): instr.cacheable_key( init_pcoll_2, instr.pcolls_to_pcoll_id(p2, ctx), - {'ref_PCollection_PCollection_10': str(id(init_pcoll))}), - str(id(init_pcoll)) + '_ref_PCollection_PCollection_10') + {'ref_PCollection_PCollection_8': str(id(init_pcoll))}), + str(id(init_pcoll)) + '_ref_PCollection_PCollection_8') def test_cache_key(self): p = beam.Pipeline(interactive_runner.InteractiveRunner()) @@ -131,19 +131,19 @@ def test_cacheables(self): pipeline_instrument._cacheable_key(init_pcoll): instr.Cacheable( var='init_pcoll', version=str(id(init_pcoll)), - pcoll_id='ref_PCollection_PCollection_10', + pcoll_id='ref_PCollection_PCollection_8', producer_version=str(id(init_pcoll.producer)), pcoll=init_pcoll), pipeline_instrument._cacheable_key(squares): instr.Cacheable( var='squares', version=str(id(squares)), - pcoll_id='ref_PCollection_PCollection_11', + pcoll_id='ref_PCollection_PCollection_9', producer_version=str(id(squares.producer)), pcoll=squares), pipeline_instrument._cacheable_key(cubes): instr.Cacheable( var='cubes', version=str(id(cubes)), - pcoll_id='ref_PCollection_PCollection_12', + pcoll_id='ref_PCollection_PCollection_10', producer_version=str(id(cubes.producer)), pcoll=cubes) }) diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py index 2a507da8c0c3..5deb4f6be44d 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py @@ -1190,7 +1190,7 @@ def has_mi_for_ptransform(mon_infos, ptransform): # postgbk monitoring infos labels = { - monitoring_infos.PCOLLECTION_LABEL: 'ref_PCollection_PCollection_8' + monitoring_infos.PCOLLECTION_LABEL: 'ref_PCollection_PCollection_6' } self.assert_has_counter( postgbk_mis, monitoring_infos.ELEMENT_COUNT_URN, labels, value=1) @@ -1198,7 +1198,7 @@ def has_mi_for_ptransform(mon_infos, ptransform): postgbk_mis, monitoring_infos.SAMPLED_BYTE_SIZE_URN, labels) labels = { - monitoring_infos.PCOLLECTION_LABEL: 'ref_PCollection_PCollection_9' + monitoring_infos.PCOLLECTION_LABEL: 'ref_PCollection_PCollection_7' } self.assert_has_counter( postgbk_mis, monitoring_infos.ELEMENT_COUNT_URN, labels, value=5) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 92b88f95e244..e1cbf97ebaf4 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2226,50 +2226,11 @@ def infer_output_type(self, input_type): key_type, typehints.WindowedValue[value_type]]] # type: ignore[misc] def expand(self, pcoll): - # This code path is only used in the local direct runner. For Dataflow - # runner execution, the GroupByKey transform is expanded on the service. - input_type = pcoll.element_type - if input_type is not None: - # Initialize type-hints used below to enforce type-checking and to pass - # downstream to further PTransforms. - key_type, value_type = trivial_inference.key_value_types(input_type) - # Enforce the input to a GBK has a KV element type. - pcoll.element_type = typehints.KV[key_type, value_type] - typecoders.registry.verify_deterministic( - typecoders.registry.get_coder(key_type), - 'GroupByKey operation "%s"' % self.label) - - reify_output_type = typehints.KV[ - key_type, typehints.WindowedValue[value_type]] # type: ignore[misc] - gbk_input_type = ( - typehints.KV[ - key_type, - typehints.Iterable[typehints.WindowedValue[ # type: ignore[misc] - value_type]]]) - gbk_output_type = typehints.KV[key_type, typehints.Iterable[value_type]] - - # pylint: disable=bad-continuation - return ( - pcoll - | 'ReifyWindows' >> - (ParDo(self.ReifyWindows()).with_output_types(reify_output_type)) - | 'GroupByKey' >> ( - _GroupByKeyOnly().with_input_types( - reify_output_type).with_output_types(gbk_input_type)) - | ( - 'GroupByWindow' >> - _GroupAlsoByWindow(pcoll.windowing).with_input_types( - gbk_input_type).with_output_types(gbk_output_type))) - else: - # The input_type is None, run the default - return ( - pcoll - | 'ReifyWindows' >> ParDo(self.ReifyWindows()) - | 'GroupByKey' >> _GroupByKeyOnly() - | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) + return pvalue.PCollection.from_(pcoll) def infer_output_type(self, input_type): - key_type, value_type = trivial_inference.key_value_types(input_type) + key_type, value_type = (typehints.typehints.coerce_to_kv_type( + input_type).tuple_types) return typehints.KV[key_type, typehints.Iterable[value_type]] def to_runner_api_parameter(self, unused_context): From 7283332a8f158f10fdaed946e42823cdfbdd1f24 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 12 May 2020 13:52:21 -0700 Subject: [PATCH 004/151] wip: add stubs for create, update write methods --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 272 +++++++++++++++++- .../gcp/healthcare/HealthcareApiClient.java | 11 +- .../healthcare/HttpHealthcareApiClient.java | 114 +++++++- 3 files changed, 378 insertions(+), 19 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 13609a5ed63d..29ec6fafc2b2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -46,12 +46,13 @@ import org.apache.beam.sdk.io.fs.MoveOptions.StandardMoveOptions; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.ConditionalUpdate.ConditionalUpdateFn; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.ExecuteBundles.ExecuteBundlesFn; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupIntoBatches; import org.apache.beam.sdk.transforms.PTransform; @@ -221,7 +222,7 @@ public static class Result implements POutput, PInput { PCollectionTuple pct; /** - * Create FhirIO.Read.Result form PCollectionTuple with OUT and DEAD_LETTER tags. + * ConditionalUpdate FhirIO.Read.Result form PCollectionTuple with OUT and DEAD_LETTER tags. * * @param pct the pct * @return the read result @@ -479,7 +480,7 @@ private Result( if (failedFiles == null) { failedFiles = (PCollection>) - pipeline.apply(Create.empty(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + pipeline.apply(org.apache.beam.sdk.transforms.Create.empty(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } this.failedFiles = failedFiles; } @@ -577,7 +578,7 @@ private static Write.Builder write(String fhirStore) { } /** - * Create Method creates a single FHIR resource. @see * * @param fhirStore the hl 7 v 2 store @@ -664,7 +665,7 @@ public Result expand(PCollection input) { input .apply( "Execute FHIR Bundles", - ParDo.of(new ExecuteBundles.ExecuteBundlesFn(this.getFhirStore()))) + ParDo.of(new ExecuteBundlesFn(this.getFhirStore()))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of())); } return Result.in(input.getPipeline(), failedBundles); @@ -675,7 +676,7 @@ public Result expand(PCollection input) { * Writes each bundle of elements to a new-line delimited JSON file on GCS and issues a * fhirStores.import Request for that file. This is intended for batch use only to facilitate * large backfills to empty FHIR stores and should not be used with unbounded PCollections. If - * your use case is streaming checkout using {@link ExecuteBundles} to more safely execute bundles + * your use case is streaming checkout using {@link ConditionalUpdate} to more safely execute bundles * as transactions which is safer practice for a use on a "live" FHIR store. */ public static class Import extends Write { @@ -796,7 +797,7 @@ public Write.Result expand(PCollection input) { // failed / rescheduled ImportFn::importBatch. input .getPipeline() - .apply(Create.of(Collections.singletonList(tempGcsPath))) + .apply(org.apache.beam.sdk.transforms.Create.of(Collections.singletonList(tempGcsPath))) .apply("Wait On File Writing", Wait.on(failedBodies)) .apply("Wait On FHIR Importing", Wait.on(failedFiles)) .apply( @@ -1104,4 +1105,261 @@ public void executeBundles(ProcessContext context) { } } } + + public static class Create extends PTransform, Write.Result> { + private final String fhirStore; + private final String type; + private final boolean ifNoneExist; + + /** + * Instantiates a new Execute bundles. + * @param fhirStore the fhir store + * @param type + * @param ifNoneExist + */ + Create(ValueProvider fhirStore, String type, boolean ifNoneExist) { + this.fhirStore = fhirStore.get(); + this.type = type; + this.ifNoneExist = ifNoneExist; + } + + /** + * Instantiates a new Execute bundles. + * + * @param fhirStore the fhir store + */ + Create(String fhirStore, String type, boolean ifNoneExist) { + this.fhirStore = fhirStore; + this.type = type; + this.ifNoneExist = ifNoneExist; + } + + @Override + public FhirIO.Write.Result expand(PCollection input) { + return Write.Result.in( + input.getPipeline(), + input + .apply(ParDo.of(new CreateFn(fhirStore, type, ifNoneExist))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + } + + /** The type Write Fhir fn. */ + static class CreateFn extends DoFn> { + + private Counter failedBundles = Metrics.counter(CreateFn.class, "failed-bundles"); + private transient HealthcareApiClient client; + private final ObjectMapper mapper = new ObjectMapper(); + /** The Fhir store. */ + private final String fhirStore; + private final String type; + private final boolean ifNoneExist; + + /** + * Instantiates a new Write Fhir fn. + * @param fhirStore the Fhir store + * @param type + * @param ifNoneExist + */ + CreateFn(String fhirStore, String type, boolean ifNoneExist) { + this.fhirStore = fhirStore; + this.type = type; + this.ifNoneExist = ifNoneExist; + } + + /** + * Initialize healthcare client. + * + * @throws IOException the io exception + */ + @Setup + public void initClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + } + + /** + * Execute Bundles. + * + * @param context the context + */ + @ProcessElement + public void create(ProcessContext context) { + String body = context.element(); + try { + // Validate that data was set to valid JSON. + mapper.readTree(body); + client.fhirCreate(fhirStore, type, body, ifNoneExist); + } catch (IOException | HealthcareHttpException e) { + failedBundles.inc(); + context.output(HealthcareIOError.of(body, e)); + } + } + } + } + + public static class Update extends PTransform, Write.Result> { + private final String fhirStore; + private final String resourceName; + + /** + * Instantiates a new Execute bundles. + * @param fhirStore the fhir store + * @param resourceName + * @param conditional + */ + Update(ValueProvider fhirStore, String resourceName, boolean conditional) { + this.fhirStore = fhirStore.get(); + this.resourceName = resourceName; + } + + /** + * Instantiates a new Execute bundles. + * + * @param fhirStore the fhir store + */ + Update(String fhirStore, String resourceName, boolean conditional) { + this.fhirStore = fhirStore; + this.resourceName = resourceName; + } + + @Override + public FhirIO.Write.Result expand(PCollection input) { + return Write.Result.in( + input.getPipeline(), + input + .apply(ParDo.of(new UpdateFn(fhirStore, resourceName))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + } + + /** The type Write Fhir fn. */ + static class UpdateFn extends DoFn> { + + private Counter failedBundles = Metrics.counter(UpdateFn.class, "failed-bundles"); + private transient HealthcareApiClient client; + private final ObjectMapper mapper = new ObjectMapper(); + /** The Fhir store. */ + private final String fhirStore; + private final String resourceName; + + /** + * Instantiates a new Write Fhir fn. + * @param fhirStore the Fhir store + * @param resourceName + */ + UpdateFn(String fhirStore, String resourceName) { + this.fhirStore = fhirStore; + this.resourceName = resourceName; + } + + /** + * Initialize healthcare client. + * + * @throws IOException the io exception + */ + @Setup + public void initClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + } + + /** + * ConditionalUpdate resources. + * + * @param context the context + */ + @ProcessElement + public void update(ProcessContext context) { + String body = context.element(); + try { + // Validate that data was set to valid JSON. + mapper.readTree(body); + client.fhirUpdate(fhirStore, resourceName, body); + } catch (IOException | HealthcareHttpException e) { + failedBundles.inc(); + context.output(HealthcareIOError.of(body, e)); + } + } + } + } + + public static class ConditionalUpdate extends PTransform, Write.Result> { + private final String fhirStore; + private final String type; + + /** + * Instantiates a new Execute bundles. + * @param fhirStore the fhir store + * @param type + */ + ConditionalUpdate(ValueProvider fhirStore, String type, boolean conditional) { + this.fhirStore = fhirStore.get(); + this.type = type; + } + + /** + * Instantiates a new Execute bundles. + * + * @param fhirStore the fhir store + */ + ConditionalUpdate(String fhirStore, String type, boolean conditional) { + this.fhirStore = fhirStore; + this.type = type; + } + + @Override + public FhirIO.Write.Result expand(PCollection input) { + return Write.Result.in( + input.getPipeline(), + input + .apply(ParDo.of(new ConditionalUpdateFn(fhirStore, type))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + } + + /** The type Write Fhir fn. */ + static class ConditionalUpdateFn extends DoFn> { + + private Counter failedBundles = Metrics.counter(ConditionalUpdateFn.class, "failed-bundles"); + private transient HealthcareApiClient client; + private final ObjectMapper mapper = new ObjectMapper(); + /** The Fhir store. */ + private final String fhirStore; + private final String type; + + /** + * Instantiates a new Write Fhir fn. + * @param fhirStore the Fhir store + * @param type + */ + ConditionalUpdateFn(String fhirStore, String type) { + this.fhirStore = fhirStore; + this.type = type; + } + + /** + * Initialize healthcare client. + * + * @throws IOException the io exception + */ + @Setup + public void initClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + } + + /** + * ConditionalUpdate resources. + * + * @param context the context + */ + @ProcessElement + public void conditionalUpdate(ProcessContext context) { + String body = context.element(); + try { + // Validate that data was set to valid JSON. + mapper.readTree(body); + client.fhirConditionalUpdate(fhirStore, type, body); + } catch (IOException | HealthcareHttpException e) { + failedBundles.inc(); + context.output(HealthcareIOError.of(body, e)); + } + } + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index 458760741915..8e35560014f3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -80,7 +80,7 @@ ListMessagesResponse makeHL7v2ListRequest( IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg) throws IOException; /** - * Create hl 7 v 2 message message. + * ConditionalUpdate hl 7 v 2 message message. * * @param hl7v2Store the hl 7 v 2 store * @param msg the msg @@ -106,6 +106,15 @@ Operation pollOperation(Operation operation, Long sleepMs) HttpBody executeFhirBundle(String fhirStore, String bundle) throws IOException, HealthcareHttpException; + HttpBody fhirCreate(String fhirStore, String type, String resource, boolean ifNoneExist) + throws IOException, HealthcareHttpException; + + HttpBody fhirConditionalUpdate(String fhirStore, String relativeResourceName, String resource) + throws IOException, HealthcareHttpException; + + HttpBody fhirUpdate(String fhirStore, String type, String resource) + throws IOException, HealthcareHttpException; + /** * Read fhir resource http body. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 909f18843949..ab3d9e965760 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -47,6 +47,7 @@ import java.net.URISyntaxException; import java.text.ParseException; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -54,6 +55,7 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.FhirHttpRequest.Method; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; import org.apache.http.HttpEntity; @@ -85,7 +87,7 @@ public class HttpHealthcareApiClient implements HealthcareApiClient, Serializabl private static final String FHIRSTORE_HEADER_ACCEPT_CHARSET = "utf-8"; private static final Logger LOG = LoggerFactory.getLogger(HttpHealthcareApiClient.class); private transient CloudHealthcare client; - private transient HttpClient httpClient; + public transient HttpClient httpClient; private transient GoogleCredentials credentials; /** @@ -283,7 +285,7 @@ public Operation pollOperation(Operation operation, Long sleepMs) throws InterruptedException, IOException { LOG.debug(String.format("started opertation %s. polling until complete.", operation.getName())); while (operation.getDone() == null || !operation.getDone()) { - // Update the status of the operation with another request. + // ConditionalUpdate the status of the operation with another request. Thread.sleep(sleepMs); // Pause between requests. operation = client.projects().locations().datasets().operations().get(operation.getName()).execute(); @@ -291,19 +293,65 @@ public Operation pollOperation(Operation operation, Long sleepMs) return operation; } - @Override - public HttpBody executeFhirBundle(String fhirStore, String bundle) - throws IOException, HealthcareHttpException { + static class FhirHttpRequest { + private final String fhirStore; + private final String payload; + private Method method; + private String pathSuffix; + private Map headers; + + + enum Method { + POST, + PUT + } + + FhirHttpRequest(String fhirStore, @Nullable String payload) { + this.fhirStore = fhirStore; + this.payload = payload; + this.method = Method.POST; + this.headers = new HashMap<>(); + } + + public static FhirHttpRequest of(String fhirStore, String payload) { + return new FhirHttpRequest(fhirStore, payload); + } + + public FhirHttpRequest setPathSuffix(String pathSuffix){ + this.pathSuffix = pathSuffix; + return this; + } + + public FhirHttpRequest setHeaders(Map headers){ + this.headers = headers; + return this; + } + + public FhirHttpRequest setMethod(Method method){ + this.method = method; + return this; + } + + + + } + + public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) throws IOException, HealthcareHttpException { if (httpClient == null || client == null) { initClient(); } credentials.refreshIfExpired(); - StringEntity requestEntity = new StringEntity(bundle, ContentType.APPLICATION_JSON); + StringEntity requestEntity = new StringEntity(fhirHttpRequest.payload, ContentType.APPLICATION_JSON); URI uri; try { + String baseFhirUri= client.getRootUrl() + "v1beta1/" + fhirHttpRequest.fhirStore + "/fhir"; + String uriString = baseFhirUri; + if (!Strings.isNullOrEmpty(fhirHttpRequest.pathSuffix)){ + uriString += fhirHttpRequest.pathSuffix; + } uri = - new URIBuilder(client.getRootUrl() + "v1beta1/" + fhirStore + "/fhir") + new URIBuilder(uriString) .setParameter("access_token", credentials.getAccessToken().getTokenValue()) .build(); } catch (URISyntaxException e) { @@ -311,8 +359,17 @@ public HttpBody executeFhirBundle(String fhirStore, String bundle) throw new IllegalArgumentException(e); } - HttpUriRequest request = - RequestBuilder.post() + RequestBuilder requestBuilder; + switch (fhirHttpRequest.method){ + case PUT: + requestBuilder = RequestBuilder.put(); + break; + case POST: + default: + requestBuilder = RequestBuilder.post(); + } + + HttpUriRequest request = requestBuilder .setUri(uri) .setEntity(requestEntity) .addHeader("User-Agent", USER_AGENT) @@ -321,6 +378,11 @@ public HttpBody executeFhirBundle(String fhirStore, String bundle) .addHeader("Accept", FHIRSTORE_HEADER_ACCEPT) .build(); + // Add headers + for (Map.Entry header: fhirHttpRequest.headers.entrySet()){ + request.addHeader(header.getKey(), header.getValue()); + } + HttpResponse response = httpClient.execute(request); HttpEntity responseEntity = response.getEntity(); String content = EntityUtils.toString(responseEntity); @@ -333,6 +395,36 @@ public HttpBody executeFhirBundle(String fhirStore, String bundle) return responseModel; } + @Override + public HttpBody executeFhirBundle(String fhirStore, String bundle) + throws IOException, HealthcareHttpException { + return executeFhirHttpRequest(FhirHttpRequest.of(fhirStore, bundle)); + } + + @Override + public HttpBody fhirCreate(String fhirStore, String type, String resource, boolean ifNoneExist) + throws IOException, HealthcareHttpException { + return executeFhirHttpRequest(FhirHttpRequest.of(fhirStore, resource).setPathSuffix("/" + type)); + } + + @Override + public HttpBody fhirConditionalUpdate(String fhirStore, String type, String resource) + throws IOException, HealthcareHttpException { + return executeFhirHttpRequest( + FhirHttpRequest.of(fhirStore, resource) + .setPathSuffix("/" + type) + .setMethod(Method.PUT)); + } + + @Override + public HttpBody fhirUpdate(String fhirStore, String relativeResourceName, String resource) + throws IOException, HealthcareHttpException { + return executeFhirHttpRequest( + FhirHttpRequest.of(fhirStore, resource) + .setPathSuffix("/" + relativeResourceName) + .setMethod(Method.PUT)); + } + /** * Wraps {@link HttpResponse} in an exception with a statusCode field for use with {@link * HealthcareIOError}. @@ -352,7 +444,7 @@ public static class HealthcareHttpException extends Exception { } /** - * Create Exception of {@link HttpResponse}. + * ConditionalUpdate Exception of {@link HttpResponse}. * * @param response the HTTP response * @return the healthcare http exception @@ -408,7 +500,7 @@ public void initialize(HttpRequest request) throws IOException { private void initClient() throws IOException { credentials = GoogleCredentials.getApplicationDefault(); - // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests. + // ConditionalUpdate a HttpRequestInitializer, which will provide a baseline configuration to all requests. HttpRequestInitializer requestInitializer = new AuthenticatedRetryInitializer( credentials.createScoped( From 2e17e06c1e41df3addaeb065e12fb6d8040f7bf3 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 12 May 2020 15:35:05 -0700 Subject: [PATCH 005/151] wip --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 54 +++++++++++-------- .../sdk/io/gcp/healthcare/HL7v2Message.java | 2 + .../gcp/healthcare/HealthcareApiClient.java | 2 +- .../healthcare/HttpHealthcareApiClient.java | 13 ++++- 4 files changed, 45 insertions(+), 26 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 29ec6fafc2b2..c546a0726ec2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -57,6 +57,7 @@ import org.apache.beam.sdk.transforms.GroupIntoBatches; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.Wait; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -1109,29 +1110,28 @@ public void executeBundles(ProcessContext context) { public static class Create extends PTransform, Write.Result> { private final String fhirStore; private final String type; - private final boolean ifNoneExist; + private SerializableFunction ifNoneExistFunction; - /** - * Instantiates a new Execute bundles. - * @param fhirStore the fhir store - * @param type - * @param ifNoneExist - */ - Create(ValueProvider fhirStore, String type, boolean ifNoneExist) { + Create(ValueProvider fhirStore, String type) { this.fhirStore = fhirStore.get(); this.type = type; - this.ifNoneExist = ifNoneExist; } - /** - * Instantiates a new Execute bundles. - * - * @param fhirStore the fhir store - */ - Create(String fhirStore, String type, boolean ifNoneExist) { + Create(String fhirStore, String type) { this.fhirStore = fhirStore; this.type = type; - this.ifNoneExist = ifNoneExist; + } + + /** This should be function that reads an resource string and extracts an + * If-None-Exists query for conditional create. + * Typically this will just be extracting an ID. + * + * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/create + * */ + public Create withIfNotExistFunction( + SerializableFunction ifNoneExistFunction){ + this.ifNoneExistFunction = ifNoneExistFunction; + return this; } @Override @@ -1139,7 +1139,7 @@ public FhirIO.Write.Result expand(PCollection input) { return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new CreateFn(fhirStore, type, ifNoneExist))) + .apply(ParDo.of(new CreateFn(fhirStore, type, ifNoneExistFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1152,20 +1152,22 @@ static class CreateFn extends DoFn> { /** The Fhir store. */ private final String fhirStore; private final String type; - private final boolean ifNoneExist; + private SerializableFunction ifNoneExistFunction; /** * Instantiates a new Write Fhir fn. - * @param fhirStore the Fhir store + * @param fhirStore the Fhir store * @param type - * @param ifNoneExist + * @param ifNoneExistFunction */ - CreateFn(String fhirStore, String type, boolean ifNoneExist) { + CreateFn(String fhirStore, String type, + SerializableFunction ifNoneExistFunction) { this.fhirStore = fhirStore; this.type = type; - this.ifNoneExist = ifNoneExist; + this.ifNoneExistFunction = ifNoneExistFunction; } + /** * Initialize healthcare client. * @@ -1187,7 +1189,12 @@ public void create(ProcessContext context) { try { // Validate that data was set to valid JSON. mapper.readTree(body); - client.fhirCreate(fhirStore, type, body, ifNoneExist); + if (ifNoneExistFunction != null){ + String ifNoneExistQuery = ifNoneExistFunction.apply(body); + client.fhirCreate(fhirStore, type, body, ifNoneExistQuery); + } else{ + client.fhirCreate(fhirStore, type, body, null); + } } catch (IOException | HealthcareHttpException e) { failedBundles.inc(); context.output(HealthcareIOError.of(body, e)); @@ -1333,6 +1340,7 @@ static class ConditionalUpdateFn extends DoFn> this.type = type; } + /** * Initialize healthcare client. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java index 8e846db5b57d..5cf3d2cdbfea 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java @@ -22,8 +22,10 @@ import java.io.IOException; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.DefaultCoder; /** The type HL7v2 message to wrap the {@link Message} model. */ +@DefaultCoder(HL7v2MessageCoder.class) public class HL7v2Message { private static final String schematizedDataKey = "schematizedData"; private static final String schematizedDataPrefix = "{data="; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index 8e35560014f3..efa594112ee4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -106,7 +106,7 @@ Operation pollOperation(Operation operation, Long sleepMs) HttpBody executeFhirBundle(String fhirStore, String bundle) throws IOException, HealthcareHttpException; - HttpBody fhirCreate(String fhirStore, String type, String resource, boolean ifNoneExist) + HttpBody fhirCreate(String fhirStore, String type, String resource, @Nullable String ifNoneExist) throws IOException, HealthcareHttpException; HttpBody fhirConditionalUpdate(String fhirStore, String relativeResourceName, String resource) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index ab3d9e965760..a8582dd2971e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -402,9 +402,18 @@ public HttpBody executeFhirBundle(String fhirStore, String bundle) } @Override - public HttpBody fhirCreate(String fhirStore, String type, String resource, boolean ifNoneExist) + public HttpBody fhirCreate(String fhirStore, String type, String resource, + @Nullable String ifNoneExist) throws IOException, HealthcareHttpException { - return executeFhirHttpRequest(FhirHttpRequest.of(fhirStore, resource).setPathSuffix("/" + type)); + Map headers = new HashMap<>(); + if (Strings.isNullOrEmpty(ifNoneExist)) { + headers.put("If-None-Exist", ifNoneExist); + } + return executeFhirHttpRequest( + FhirHttpRequest.of(fhirStore, resource) + .setPathSuffix("/" + type) + .setHeaders(headers) + ); } @Override From 5510822b0b3928c44af98853c12b870a32ca08d6 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Wed, 13 May 2020 10:59:48 -0700 Subject: [PATCH 006/151] remove unused method from FhirIOTest --- .../sdk/io/gcp/healthcare/FhirIOTest.java | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java index 976c55fcb76c..f06e27cd3017 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java @@ -88,25 +88,4 @@ public void test_FhirIO_failedWrites() { private static final long NUM_ELEMENTS = 11; private static final long BATCH_SIZE = 5; - private static ArrayList> createTestData() { - String[] scientists = { - "Einstein", - "Darwin", - "Copernicus", - "Pasteur", - "Curie", - "Faraday", - "Newton", - "Bohr", - "Galilei", - "Maxwell" - }; - ArrayList> data = new ArrayList<>(); - for (int i = 0; i < NUM_ELEMENTS; i++) { - int index = i % scientists.length; - KV element = KV.of("key", scientists[index]); - data.add(element); - } - return data; - } } From 986de56ec3178ea85fc6ac5c1bab56ec920e60a1 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 13 May 2020 14:14:36 -0700 Subject: [PATCH 007/151] Move ReifyWindows etc to DirectRunner, don't subclass from GBK Change-Id: I9a54cbaadf3968dbb48167cc669088212680d336 --- .../runners/dataflow/dataflow_runner_test.py | 4 + .../runners/direct/direct_runner.py | 194 +++++++++++------- .../runners/direct/transform_evaluator.py | 5 +- sdks/python/apache_beam/transforms/core.py | 68 ------ 4 files changed, 132 insertions(+), 139 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 0791b49a983d..b09f38d3427d 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -396,6 +396,10 @@ def test_gbk_then_flatten_input_visitor(self): flat = (none_str_pc, none_int_pc) | beam.Flatten() _ = flat | beam.GroupByKey() + # This may change if type inference changes, but we assert it here + # to make sure the check below is not vacuous. + self.assertNotIsInstance(flat.element_type, typehints.TupleConstraint) + p.visit(DataflowRunner.group_by_key_input_visitor()) p.visit(DataflowRunner.flatten_input_visitor()) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index fb80f19b5d3b..8fa6c902900d 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -50,10 +50,10 @@ from apache_beam.transforms.core import CombineValuesDoFn from apache_beam.transforms.core import DoFn from apache_beam.transforms.core import ParDo -from apache_beam.transforms.core import _GroupAlsoByWindow -from apache_beam.transforms.core import _GroupAlsoByWindowDoFn -from apache_beam.transforms.core import _GroupByKeyOnly from apache_beam.transforms.ptransform import PTransform +from apache_beam.transforms.window import WindowedValue +from apache_beam.typehints import trivial_inference +from apache_beam.typehints.decorators import TypeCheckError # Note that the BundleBasedDirectRunner and SwitchingDirectRunner names are # experimental and have no backwards compatibility guarantees. @@ -135,6 +135,57 @@ def visit_transform(self, applied_ptransform): V = typing.TypeVar('V') +@typehints.with_input_types(typing.Tuple[K, V]) +@typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) +class _GroupByKeyOnly(PTransform): + """A group by key transform, ignoring windows.""" + def infer_output_type(self, input_type): + key_type, value_type = trivial_inference.key_value_types(input_type) + return typehints.KV[key_type, typehints.Iterable[value_type]] + + def expand(self, pcoll): + self._check_pcollection(pcoll) + return PCollection.from_(pcoll) + + +@typehints.with_input_types(typing.Tuple[K, typing.Iterable[V]]) +@typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) +class _GroupAlsoByWindow(ParDo): + """The GroupAlsoByWindow transform.""" + def __init__(self, windowing): + super(_GroupAlsoByWindow, self).__init__(_GroupAlsoByWindowDoFn(windowing)) + self.windowing = windowing + + def expand(self, pcoll): + self._check_pcollection(pcoll) + return PCollection.from_(pcoll) + + +class _GroupAlsoByWindowDoFn(DoFn): + # TODO(robertwb): Support combiner lifting. + + def __init__(self, windowing): + super(_GroupAlsoByWindowDoFn, self).__init__() + self.windowing = windowing + + def infer_output_type(self, input_type): + key_type, windowed_value_iter_type = trivial_inference.key_value_types( + input_type) + value_type = windowed_value_iter_type.inner_type.inner_type + return typehints.Iterable[typehints.KV[key_type, + typehints.Iterable[value_type]]] + + def start_bundle(self): + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.transforms.trigger import create_trigger_driver + # pylint: enable=wrong-import-order, wrong-import-position + self.driver = create_trigger_driver(self.windowing, True) + + def process(self, element): + k, vs = element + return self.driver.process_entire_key(k, vs) + + @typehints.with_input_types(typing.Tuple[K, V]) @typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) class _StreamingGroupByKeyOnly(_GroupByKeyOnly): @@ -172,6 +223,75 @@ def from_runner_api_parameter(unused_ptransform, payload, context): context.windowing_strategies.get_by_id(payload.value)) +@typehints.with_input_types(typing.Tuple[K, typing.Iterable[V]]) +@typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) +class _GroupByKey(PTransform): + """The DirectRunner GroupByKey implementation.""" + class ReifyWindows(DoFn): + def process( + self, element, window=DoFn.WindowParam, timestamp=DoFn.TimestampParam): + try: + k, v = element + except TypeError: + raise TypeCheckError( + 'Input to GroupByKey must be a PCollection with ' + 'elements compatible with KV[A, B]') + + return [(k, WindowedValue(v, timestamp, [window]))] + + def infer_output_type(self, input_type): + key_type, value_type = trivial_inference.key_value_types(input_type) + return typehints.Iterable[typehints.KV[ + key_type, typehints.WindowedValue[value_type]]] # type: ignore[misc] + + def expand(self, pcoll): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.coders import typecoders + + input_type = pcoll.element_type + if input_type is not None: + # Initialize type-hints used below to enforce type-checking and to + # pass downstream to further PTransforms. + key_type, value_type = trivial_inference.key_value_types(input_type) + # Enforce the input to a GBK has a KV element type. + pcoll.element_type = typehints.typehints.coerce_to_kv_type( + pcoll.element_type) + typecoders.registry.verify_deterministic( + typecoders.registry.get_coder(key_type), + 'GroupByKey operation "%s"' % self.label) + + reify_output_type = typehints.KV[ + key_type, typehints.WindowedValue[value_type]] # type: ignore[misc] + gbk_input_type = ( + typehints. + KV[key_type, + typehints.Iterable[ + typehints.WindowedValue[ # type: ignore[misc] + value_type]]]) + gbk_output_type = typehints.KV[key_type, typehints.Iterable[value_type]] + + # pylint: disable=bad-continuation + return ( + pcoll + | 'ReifyWindows' >> + (ParDo(self.ReifyWindows()).with_output_types(reify_output_type)) + | 'GroupByKey' >> ( + _GroupByKeyOnly().with_input_types( + reify_output_type).with_output_types(gbk_input_type)) + | ( + 'GroupByWindow' >> + _GroupAlsoByWindow(pcoll.windowing).with_input_types( + gbk_input_type).with_output_types(gbk_output_type))) + else: + # The input_type is None, run the default + return ( + pcoll + | 'ReifyWindows' >> ParDo(self.ReifyWindows()) + | 'GroupByKey' >> _GroupByKeyOnly() + | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) + + def _get_transform_overrides(pipeline_options): # A list of PTransformOverride objects to be applied before running a pipeline # using DirectRunner. @@ -243,74 +363,10 @@ def matches(self, applied_ptransform): # Imported here to avoid circular dependencies. # pylint: disable=wrong-import-order, wrong-import-position from apache_beam.transforms.core import GroupByKey - if (isinstance(applied_ptransform.transform, GroupByKey) and - not getattr(applied_ptransform.transform, 'override', False)): - self.input_type = applied_ptransform.inputs[0].element_type - return True - return False + return isinstance(applied_ptransform.transform, GroupByKey) def get_replacement_transform(self, ptransform): - # Imported here to avoid circular dependencies. - # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.transforms.core import GroupByKey - - # Subclass from GroupByKey to inherit all the proper methods. - class GroupByKey(GroupByKey): - override = True - - def expand(self, pcoll): - # Imported here to avoid circular dependencies. - # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.coders import typecoders - from apache_beam.typehints import trivial_inference - - input_type = pcoll.element_type - if input_type is not None: - # Initialize type-hints used below to enforce type-checking and to - # pass downstream to further PTransforms. - key_type, value_type = trivial_inference.key_value_types(input_type) - # Enforce the input to a GBK has a KV element type. - pcoll.element_type = typehints.typehints.coerce_to_kv_type( - pcoll.element_type) - typecoders.registry.verify_deterministic( - typecoders.registry.get_coder(key_type), - 'GroupByKey operation "%s"' % self.label) - - reify_output_type = typehints.KV[ - key_type, typehints.WindowedValue[value_type]] # type: ignore[misc] - gbk_input_type = ( - typehints. - KV[key_type, - typehints.Iterable[ - typehints.WindowedValue[ # type: ignore[misc] - value_type]]]) - gbk_output_type = typehints.KV[key_type, - typehints.Iterable[value_type]] - - # pylint: disable=bad-continuation - return ( - pcoll - | 'ReifyWindows' >> ( - ParDo(self.ReifyWindows()).with_output_types( - reify_output_type)) - | 'GroupByKey' >> ( - _GroupByKeyOnly().with_input_types( - reify_output_type).with_output_types(gbk_input_type)) - | ( - 'GroupByWindow' >> - _GroupAlsoByWindow(pcoll.windowing).with_input_types( - gbk_input_type).with_output_types(gbk_output_type))) - else: - # The input_type is None, run the default - return ( - pcoll - | 'ReifyWindows' >> ParDo(self.ReifyWindows()) - | 'GroupByKey' >> _GroupByKeyOnly() - | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) - - # Infer the correct type. - return GroupByKey().with_output_types( - ptransform.infer_output_type(self.input_type)) + return _GroupByKey() overrides = [ # This needs to be the first and the last override. Other overrides depend diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index bc6f397968ba..6421b3935e6c 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -45,6 +45,7 @@ from apache_beam.runners.common import DoFnState from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access from apache_beam.runners.direct.direct_runner import _DirectReadFromPubSub +from apache_beam.runners.direct.direct_runner import _GroupByKeyOnly from apache_beam.runners.direct.direct_runner import _StreamingGroupAlsoByWindow from apache_beam.runners.direct.direct_runner import _StreamingGroupByKeyOnly from apache_beam.runners.direct.direct_userstate import DirectUserStateContext @@ -106,7 +107,7 @@ def __init__(self, evaluation_context): core.Flatten: _FlattenEvaluator, core.Impulse: _ImpulseEvaluator, core.ParDo: _ParDoEvaluator, - core._GroupByKeyOnly: _GroupByKeyOnlyEvaluator, + _GroupByKeyOnly: _GroupByKeyOnlyEvaluator, _StreamingGroupByKeyOnly: _StreamingGroupByKeyOnlyEvaluator, _StreamingGroupAlsoByWindow: _StreamingGroupAlsoByWindowEvaluator, _NativeWrite: _NativeWriteEvaluator, @@ -176,7 +177,7 @@ def should_execute_serially(self, applied_ptransform): True if executor should execute applied_ptransform serially. """ if isinstance(applied_ptransform.transform, - (core._GroupByKeyOnly, + (_GroupByKeyOnly, _StreamingGroupByKeyOnly, _StreamingGroupAlsoByWindow, _NativeWrite)): diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index e1cbf97ebaf4..665d7ec4ebea 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2208,23 +2208,6 @@ class GroupByKey(PTransform): The implementation here is used only when run on the local direct runner. """ - class ReifyWindows(DoFn): - def process( - self, element, window=DoFn.WindowParam, timestamp=DoFn.TimestampParam): - try: - k, v = element - except TypeError: - raise TypeCheckError( - 'Input to GroupByKey must be a PCollection with ' - 'elements compatible with KV[A, B]') - - return [(k, WindowedValue(v, timestamp, [window]))] - - def infer_output_type(self, input_type): - key_type, value_type = trivial_inference.key_value_types(input_type) - return typehints.Iterable[typehints.KV[ - key_type, typehints.WindowedValue[value_type]]] # type: ignore[misc] - def expand(self, pcoll): return pvalue.PCollection.from_(pcoll) @@ -2247,57 +2230,6 @@ def runner_api_requires_keyed_input(self): return True -@typehints.with_input_types(typing.Tuple[K, V]) -@typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) -class _GroupByKeyOnly(PTransform): - """A group by key transform, ignoring windows.""" - def infer_output_type(self, input_type): - key_type, value_type = trivial_inference.key_value_types(input_type) - return typehints.KV[key_type, typehints.Iterable[value_type]] - - def expand(self, pcoll): - self._check_pcollection(pcoll) - return pvalue.PCollection.from_(pcoll) - - -@typehints.with_input_types(typing.Tuple[K, typing.Iterable[V]]) -@typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) -class _GroupAlsoByWindow(ParDo): - """The GroupAlsoByWindow transform.""" - def __init__(self, windowing): - super(_GroupAlsoByWindow, self).__init__(_GroupAlsoByWindowDoFn(windowing)) - self.windowing = windowing - - def expand(self, pcoll): - self._check_pcollection(pcoll) - return pvalue.PCollection.from_(pcoll) - - -class _GroupAlsoByWindowDoFn(DoFn): - # TODO(robertwb): Support combiner lifting. - - def __init__(self, windowing): - super(_GroupAlsoByWindowDoFn, self).__init__() - self.windowing = windowing - - def infer_output_type(self, input_type): - key_type, windowed_value_iter_type = trivial_inference.key_value_types( - input_type) - value_type = windowed_value_iter_type.inner_type.inner_type - return typehints.Iterable[typehints.KV[key_type, - typehints.Iterable[value_type]]] - - def start_bundle(self): - # pylint: disable=wrong-import-order, wrong-import-position - from apache_beam.transforms.trigger import create_trigger_driver - # pylint: enable=wrong-import-order, wrong-import-position - self.driver = create_trigger_driver(self.windowing, True) - - def process(self, element): - k, vs = element - return self.driver.process_entire_key(k, vs) - - class Partition(PTransformWithSideInputs): """Split a PCollection into several partitions. From 04f256a2c455a5dc3f319c1d0ba9304530774a2b Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Wed, 13 May 2020 14:22:47 -0700 Subject: [PATCH 008/151] draft conditional create and update transforms --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 401 ++++++++++++------ .../gcp/healthcare/HealthcareApiClient.java | 4 +- .../healthcare/HttpHealthcareApiClient.java | 42 +- .../sdk/io/gcp/healthcare/FhirIOWriteIT.java | 2 + 4 files changed, 306 insertions(+), 143 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 38c257c224b4..3ad15ee78515 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.healthcare; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.fasterxml.jackson.databind.ObjectMapper; @@ -84,49 +85,43 @@ * *

Reading

* - *

FHIR resources can be read with {@link FhirIO.Read}, which supports use cases where you have a - * ${@link PCollection} of message IDS. This is appropriate for reading the Fhir notifications from - * a Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually - * prepared list of messages that you need to process (e.g. in a text file read with {@link - * org.apache.beam.sdk.io.TextIO}*) . + *

FHIR resources can be read with {@link FhirIO.Read}, which supports use cases where you have + * a ${@link PCollection} of message IDS. This is appropriate for reading the Fhir notifications + * from a Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a + * manually prepared list of messages that you need to process (e.g. in a text file read with {@link + * org.apache.beam.sdk.io.TextIO}**) . * - *

Fetch Resource contents from Fhir Store based on the {@link PCollection} of message ID strings - * {@link FhirIO.Read.Result} where one can call {@link Read.Result#getResources()} to retrieve a - * {@link PCollection} containing the successfully fetched {@link String}s and/or {@link - * FhirIO.Read.Result#getFailedReads()}* to retrieve a {@link PCollection} of {@link - * HealthcareIOError}* containing the resource ID that could not be fetched and the exception as a + *

Fetch Resource contents from Fhir Store based on the {@link PCollection} of message ID + * strings {@link FhirIO.Read.Result} where one can call {@link Read.Result#getResources()} to + * retrieve a {@link PCollection} containing the successfully fetched {@link String}s and/or {@link + * FhirIO.Read.Result#getFailedReads()}** to retrieve a {@link PCollection} of {@link + * HealthcareIOError}** containing the resource ID that could not be fetched and the exception as a * {@link HealthcareIOError}, this can be used to write to the dead letter storage system of your * choosing. This error handling is mainly to transparently surface errors where the upstream {@link - * PCollection}* contains IDs that are not valid or are not reachable due to permissions issues. + * PCollection}** contains IDs that are not valid or are not reachable due to permissions issues. * *

Writing

* *

Write Resources can be written to FHIR with two different methods: Import or Execute Bundle. * - *

Execute Bundle This is best for use cases where you are writing to a non-empty FHIR store with - * other clients or otherwise need referential integrity (e.g. A Streaming HL7v2 to FHIR ETL + *

Execute Bundle This is best for use cases where you are writing to a non-empty FHIR store + * with other clients or otherwise need referential integrity (e.g. A Streaming HL7v2 to FHIR ETL * pipeline). * - * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/executeBundle> - *

Import This is best for use cases where you are populating an empty FHIR store with no - * other clients. It is faster than the execute bundles method but does not respect referential - * integrity and the resources are not written transactionally (e.g. a historicaly backfill on a - * new FHIR store) This requires each resource to contain a client provided ID. It is important - * that when using import you give the appropriate permissions to the Google Cloud Healthcare - * Service Agent - * @see https://cloud.google.com/healthcare/docs/how-tos/permissions-healthcare-api-gcp-products#fhir_store_cloud_storage_permissions> - * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores/import> - * A {@link PCollection} of {@link String} can be ingested into an Fhir store using {@link - * FhirIO.Write#fhirStoresImport(String, String, String, FhirIO.Import.ContentStructure)} This - * will return a {@link FhirIO.Write.Result} on which you can call {@link - * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link - * HealthcareIOError} containing the {@link String} that failed to be ingested and the - * exception. - *

Example - *

{@code
+ * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/executeBundle>
+ * 

Import This is best for use cases where you are populating an empty FHIR store with no + * other clients. It is faster than the execute bundles method but does not respect referential + * integrity and the resources are not written transactionally (e.g. a historicaly backfill on a new + * FHIR store) This requires each resource to contain a client provided ID. It is important that + * when using import you give the appropriate permissions to the Google Cloud Healthcare Service + * Agent + * @see https://cloud.google.com/healthcare/docs/how-tos/permissions-healthcare-api-gcp-products#fhir_store_cloud_storage_permissions> + * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores/import> + * A {@link PCollection} of {@link String} can be ingested into an Fhir store using {@link + * FhirIO.Write#fhirStoresImport(String, String, String, FhirIO.Import.ContentStructure)} This will + * return a {@link FhirIO.Write.Result} on which you can call {@link + * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link + * HealthcareIOError} containing the {@link String} that failed to be ingested and the exception.

Example

{@code
  * Pipeline pipeline = ...
  *
  * // Tail the FHIR store by retrieving resources based on Pub/Sub notifications.
@@ -161,8 +156,7 @@
  * // Alternatively you could use import for high throughput to a new store.
  * FhirIO.Write.Result writeResult =
  *     output.apply("Import FHIR Resources", FhirIO.executeBundles(options.getNewFhirStore()));
- * }***
- * 
+ * }***
*/ public class FhirIO { @@ -213,19 +207,27 @@ public static Import importResources( return new Import(fhirStore, tempDir, deadLetterDir, contentStructure); } - /** The type Read. */ + /** + * The type Read. + */ public static class Read extends PTransform, FhirIO.Read.Result> { private static final Logger LOG = LoggerFactory.getLogger(Read.class); - /** Instantiates a new Read. */ + /** + * Instantiates a new Read. + */ public Read() {} - /** The type Result. */ + /** + * The type Result. + */ public static class Result implements POutput, PInput { private PCollection resources; private PCollection> failedReads; - /** The Pct. */ + /** + * The Pct. + */ PCollectionTuple pct; /** @@ -287,9 +289,13 @@ public void finishSpecifyingOutput( String transformName, PInput input, PTransform transform) {} } - /** The tag for the main output of Fhir Messages. */ + /** + * The tag for the main output of Fhir Messages. + */ public static final TupleTag OUT = new TupleTag() {}; - /** The tag for the deadletter output of Fhir Messages. */ + /** + * The tag for the deadletter output of Fhir Messages. + */ public static final TupleTag> DEAD_LETTER = new TupleTag>() {}; @@ -304,7 +310,7 @@ public FhirIO.Read.Result expand(PCollection input) { *

This DoFn consumes a {@link PCollection} of notifications {@link String}s from the FHIR * store, and fetches the actual {@link String} object based on the id in the notification and * will output a {@link PCollectionTuple} which contains the output and dead-letter {@link - * PCollection}*. + * PCollection}**. * *

The {@link PCollectionTuple} output will contain the following {@link PCollection}: * @@ -312,14 +318,16 @@ public FhirIO.Read.Result expand(PCollection input) { *

  • {@link FhirIO.Read#OUT} - Contains all {@link PCollection} records successfully read * from the Fhir store. *
  • {@link FhirIO.Read#DEAD_LETTER} - Contains all {@link PCollection} of {@link - * HealthcareIOError}* of message IDs which failed to be fetched from the Fhir store, with + * HealthcareIOError}** of message IDs which failed to be fetched from the Fhir store, with * error message and stacktrace. * */ static class FetchResourceJsonString extends PTransform, FhirIO.Read.Result> { - /** Instantiates a new Fetch Fhir message DoFn. */ + /** + * Instantiates a new Fetch Fhir message DoFn. + */ public FetchResourceJsonString() {} @Override @@ -330,7 +338,9 @@ public FhirIO.Read.Result expand(PCollection resourceIds) { .withOutputTags(FhirIO.Read.OUT, TupleTagList.of(FhirIO.Read.DEAD_LETTER)))); } - /** DoFn for fetching messages from the Fhir store with error handling. */ + /** + * DoFn for fetching messages from the Fhir store with error handling. + */ static class ReadResourceFn extends DoFn { private Counter failedMessageGets = @@ -341,7 +351,9 @@ static class ReadResourceFn extends DoFn { private HealthcareApiClient client; private ObjectMapper mapper; - /** Instantiates a new Hl 7 v 2 message get fn. */ + /** + * Instantiates a new Hl 7 v 2 message get fn. + */ ReadResourceFn() {} /** @@ -392,20 +404,30 @@ private String fetchResource(HealthcareApiClient client, String resourceId) } } - /** The type Write. */ + /** + * The type Write. + */ @AutoValue public abstract static class Write extends PTransform, Write.Result> { - /** The tag for the failed writes to FHIR store`. */ + /** + * The tag for the failed writes to FHIR store`. + */ public static final TupleTag> FAILED_BODY = new TupleTag>() {}; - /** The tag for the files that failed to FHIR store`. */ + /** + * The tag for the files that failed to FHIR store`. + */ public static final TupleTag> FAILED_FILES = new TupleTag>() {}; - /** The tag for temp files for import to FHIR store`. */ + /** + * The tag for temp files for import to FHIR store`. + */ public static final TupleTag TEMP_FILES = new TupleTag() {}; - /** The enum Write method. */ + /** + * The enum Write method. + */ public enum WriteMethod { /** * Execute Bundle Method executes a batch of requests as a single transaction @see . + * FHIR stores. . */ IMPORT } - /** The type Result. */ + /** + * The type Result. + */ public static class Result implements POutput { private final Pipeline pipeline; private final PCollection> failedBodies; @@ -430,6 +453,7 @@ public static class Result implements POutput { * Creates a {@link FhirIO.Write.Result} in the given {@link Pipeline}. @param pipeline the * pipeline * + * @param pipeline the pipeline * @param failedBodies the failed inserts * @return the result */ @@ -437,6 +461,14 @@ static Result in(Pipeline pipeline, PCollection> faile return new Result(pipeline, failedBodies, null); } + /** + * In result. + * + * @param pipeline the pipeline + * @param failedBodies the failed bodies + * @param failedFiles the failed files + * @return the result + */ static Result in( Pipeline pipeline, PCollection> failedBodies, @@ -526,7 +558,9 @@ private Result( */ abstract Optional getImportGcsDeadLetterPath(); - /** The type Builder. */ + /** + * The type Builder. + */ @AutoValue.Builder abstract static class Builder { @@ -606,6 +640,14 @@ public static Write fhirStoresImport( .build(); } + /** + * Fhir stores import write. + * + * @param fhirStore the fhir store + * @param gcsDeadLetterPath the gcs dead letter path + * @param contentStructure the content structure + * @return the write + */ public static Write fhirStoresImport( String fhirStore, String gcsDeadLetterPath, @@ -618,6 +660,15 @@ public static Write fhirStoresImport( .build(); } + /** + * Fhir stores import write. + * + * @param fhirStore the fhir store + * @param gcsTempPath the gcs temp path + * @param gcsDeadLetterPath the gcs dead letter path + * @param contentStructure the content structure + * @return the write + */ public static Write fhirStoresImport( ValueProvider fhirStore, ValueProvider gcsTempPath, @@ -694,8 +745,8 @@ public Result expand(PCollection input) { * Writes each bundle of elements to a new-line delimited JSON file on GCS and issues a * fhirStores.import Request for that file. This is intended for batch use only to facilitate * large backfills to empty FHIR stores and should not be used with unbounded PCollections. If - * your use case is streaming checkout using {@link ConditionalUpdate} to more safely execute bundles - * as transactions which is safer practice for a use on a "live" FHIR store. + * your use case is streaming checkout using {@link ConditionalUpdate} to more safely execute + * bundles as transactions which is safer practice for a use on a "live" FHIR store. */ public static class Import extends Write { @@ -729,6 +780,13 @@ public static class Import extends Write { } } + /** + * Instantiates a new Import. + * + * @param fhirStore the fhir store + * @param deadLetterGcsPath the dead letter gcs path + * @param contentStructure the content structure + */ Import( ValueProvider fhirStore, ValueProvider deadLetterGcsPath, @@ -741,6 +799,7 @@ public static class Import extends Write { this.contentStructure = contentStructure; } } + /** * Instantiates a new Import. * @@ -863,7 +922,9 @@ public void delete(@Element Metadata path) { return Write.Result.in(input.getPipeline(), failedBodies, failedFiles); } - /** The Write bundles to new line delimited json files. */ + /** + * The Write bundles to new line delimited json files. + */ static class WriteBundlesToFilesFn extends DoFn { private final String fhirStore; @@ -921,6 +982,7 @@ public void initFile() throws IOException { * Add to batch. * * @param context the context + * @param window the window * @throws IOException the io exception */ @ProcessElement @@ -958,7 +1020,9 @@ public void closeFile(FinishBundleContext context) throws IOException { } } - /** Import batches of new line delimited json files to FHIR Store. */ + /** + * Import batches of new line delimited json files to FHIR Store. + */ static class ImportFn extends DoFn>, HealthcareIOError> { @@ -970,6 +1034,14 @@ static class ImportFn private HealthcareApiClient client; private final String fhirStore; + /** + * Instantiates a new Import fn. + * + * @param fhirStore the fhir store + * @param tempGcsPath the temp gcs path + * @param deadLetterGcsPath the dead letter gcs path + * @param contentStructure the content structure + */ ImportFn( String fhirStore, String tempGcsPath, @@ -985,6 +1057,11 @@ static class ImportFn } } + /** + * Init. + * + * @throws IOException the io exception + */ @Setup public void init() throws IOException { tempDir = @@ -998,6 +1075,10 @@ public void init() throws IOException { /** * Move files to a temporary subdir (to provide common prefix) to execute import with single * GCS URI. + * + * @param element the element + * @param output the output + * @throws IOException the io exception */ @ProcessElement public void importBatch( @@ -1043,9 +1124,13 @@ public void importBatch( } } - /** The enum Content structure. */ + /** + * The enum Content structure. + */ public enum ContentStructure { - /** If the content structure is not specified, the default value BUNDLE will be used. */ + /** + * If the content structure is not specified, the default value BUNDLE will be used. + */ CONTENT_STRUCTURE_UNSPECIFIED, /** * The source file contains one or more lines of newline-delimited JSON (ndjson). Each line is @@ -1058,14 +1143,20 @@ public enum ContentStructure { * a single resource. */ RESOURCE, - /** The entire file is one JSON bundle. The JSON can span multiple lines. */ + /** + * The entire file is one JSON bundle. The JSON can span multiple lines. + */ BUNDLE_PRETTY, - /** The entire file is one JSON resource. The JSON can span multiple lines. */ + /** + * The entire file is one JSON resource. The JSON can span multiple lines. + */ RESOURCE_PRETTY } } - /** The type Execute bundles. */ + /** + * The type Execute bundles. + */ public static class ExecuteBundles extends PTransform, Write.Result> { private final String fhirStore; @@ -1096,7 +1187,9 @@ public FhirIO.Write.Result expand(PCollection input) { .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** The type Write Fhir fn. */ + /** + * The type Write Fhir fn. + */ static class ExecuteBundlesFn extends DoFn> { private Counter failedBundles = Metrics.counter(ExecuteBundlesFn.class, "failed-bundles"); @@ -1144,27 +1237,47 @@ public void executeBundles(ProcessContext context) { } } + /** + * {@link PTransform} for Creating FHIR resources. + * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/create + */ public static class CreateResources extends PTransform, Write.Result> { private final String fhirStore; private final String type; - private SerializableFunction ifNoneExistFunction; + private SerializableFunction ifNoneExistFunction; + /** + * Instantiates a new Create resources transform. + * + * @param fhirStore the fhir store + * @param type the type + */ CreateResources(ValueProvider fhirStore, String type) { this.fhirStore = fhirStore.get(); this.type = type; } + /** + * Instantiates a new Create resources. + * + * @param fhirStore the fhir store + * @param type the type + */ CreateResources(String fhirStore, String type) { this.fhirStore = fhirStore; this.type = type; } - /** This should be function that reads an resource string and extracts an - * If-None-Exists query for conditional create. - * Typically this will just be extracting an ID. + /** + * This adds a {@link SerializableFunction} that reads an resource string and extracts an + * If-None-Exists query for conditional create. Typically this will just be extracting an ID + * to look for. * * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/create - * */ + * + * @param ifNoneExistFunction the if none exist function + * @return the create resources + */ public CreateResources withIfNotExistFunction( SerializableFunction ifNoneExistFunction){ this.ifNoneExistFunction = ifNoneExistFunction; @@ -1180,23 +1293,15 @@ public FhirIO.Write.Result expand(PCollection input) { .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** The type Write Fhir fn. */ static class CreateFn extends DoFn> { private Counter failedBundles = Metrics.counter(CreateFn.class, "failed-bundles"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); - /** The Fhir store. */ private final String fhirStore; private final String type; private SerializableFunction ifNoneExistFunction; - /** - * Instantiates a new Write Fhir fn. - * @param fhirStore the Fhir store - * @param type - * @param ifNoneExistFunction - */ CreateFn(String fhirStore, String type, SerializableFunction ifNoneExistFunction) { this.fhirStore = fhirStore; @@ -1204,22 +1309,11 @@ static class CreateFn extends DoFn> { this.ifNoneExistFunction = ifNoneExistFunction; } - - /** - * Initialize healthcare client. - * - * @throws IOException the io exception - */ @Setup public void initClient() throws IOException { this.client = new HttpHealthcareApiClient(); } - /** - * Execute Bundles. - * - * @param context the context - */ @ProcessElement public void create(ProcessContext context) { String body = context.element(); @@ -1240,62 +1334,80 @@ public void create(ProcessContext context) { } } + /** + * {@link PTransform} for Updating FHIR resources resources. + * + * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/update + */ public static class UpdateResources extends PTransform, Write.Result> { private final String fhirStore; - private final String resourceName; + private SerializableFunction resourceNameFunction; + public static final TupleTag UPDATE_RESULTS = new TupleTag() {}; /** - * Instantiates a new Execute bundles. - * @param fhirStore the fhir store - * @param resourceName - * @param conditional + * Instantiates a new Update resources. + * + * @param fhirStore the fhir store */ - UpdateResources(ValueProvider fhirStore, String resourceName, boolean conditional) { + UpdateResources(ValueProvider fhirStore) { this.fhirStore = fhirStore.get(); - this.resourceName = resourceName; } /** - * Instantiates a new Execute bundles. + * Add a {@link SerializableFunction} to extract a resource name from the input element. + * + * @param resourceNameFunction the resource name function + * @return the update resources + */ + public UpdateResources withResourceNameFunction( + SerializableFunction resourceNameFunction){ + this.resourceNameFunction = resourceNameFunction; + return this; + } + + /** + * Instantiates a new Update resources. * * @param fhirStore the fhir store */ - UpdateResources(String fhirStore, String resourceName, boolean conditional) { + UpdateResources(String fhirStore){ this.fhirStore = fhirStore; - this.resourceName = resourceName; } @Override public FhirIO.Write.Result expand(PCollection input) { + checkArgument(resourceNameFunction == null, + "resourceNameFunction must be set when using FhirIO.UpdateResources"); return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new UpdateFn(fhirStore, resourceName))) + .apply(ParDo.of(new UpdateFn(fhirStore, resourceNameFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** The type Write Fhir fn. */ + /** + * The type Update fn. + */ static class UpdateFn extends DoFn> { - private Counter failedBundles = Metrics.counter(UpdateFn.class, "failed-bundles"); + private Counter failedUpdates = Metrics.counter(UpdateFn.class, "failed-updates"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); - /** The Fhir store. */ private final String fhirStore; - private final String resourceName; + private SerializableFunction resourceNameFunction; /** - * Instantiates a new Write Fhir fn. - * @param fhirStore the Fhir store - * @param resourceName + * Instantiates a new Update fn. + * + * @param fhirStore the fhir store */ - UpdateFn(String fhirStore, String resourceName) { + UpdateFn(String fhirStore, SerializableFunction resourceNameFunction) { this.fhirStore = fhirStore; - this.resourceName = resourceName; + this.resourceNameFunction = resourceNameFunction; } /** - * Initialize healthcare client. + * Init client. * * @throws IOException the io exception */ @@ -1305,7 +1417,7 @@ public void initClient() throws IOException { } /** - * ConditionalUpdate resources. + * Update. * * @param context the context */ @@ -1315,23 +1427,31 @@ public void update(ProcessContext context) { try { // Validate that data was set to valid JSON. mapper.readTree(body); - client.fhirUpdate(fhirStore, resourceName, body); + String resourceName = resourceNameFunction.apply(body); + HttpBody result = client.fhirUpdate(fhirStore, resourceName, body); + context.output(UPDATE_RESULTS, result.getData()); } catch (IOException | HealthcareHttpException e) { - failedBundles.inc(); + failedUpdates.inc(); context.output(HealthcareIOError.of(body, e)); } } } } + /** + * The type Conditional update. + */ public static class ConditionalUpdate extends PTransform, Write.Result> { private final String fhirStore; private final String type; + private SerializableFunction searchParametersFunction; /** - * Instantiates a new Execute bundles. - * @param fhirStore the fhir store - * @param type + * Instantiates a new Conditional update. + * + * @param fhirStore the fhir store + * @param type the type + * @param conditional the conditional */ ConditionalUpdate(ValueProvider fhirStore, String type, boolean conditional) { this.fhirStore = fhirStore.get(); @@ -1339,9 +1459,23 @@ public static class ConditionalUpdate extends PTransform, Wr } /** - * Instantiates a new Execute bundles. + * With search parameters function conditional update. + * + * @param searchParametersFunction the search parameters function + * @return the conditional update + */ + public ConditionalUpdate withSearchParametersFunction( + SerializableFunction> searchParametersFunction){ + this.searchParametersFunction = searchParametersFunction; + return this; + } + + /** + * Instantiates a new Conditional update. * * @param fhirStore the fhir store + * @param type the type + * @param conditional the conditional */ ConditionalUpdate(String fhirStore, String type, boolean conditional) { this.fhirStore = fhirStore; @@ -1350,36 +1484,46 @@ public static class ConditionalUpdate extends PTransform, Wr @Override public FhirIO.Write.Result expand(PCollection input) { + if (searchParametersFunction == null){ + throw new IllegalArgumentException( + "FhirIO.ConditionalUpdate should always be called with a searchParametersFunction." + + "If this does not meet your use case consider usiing FhirIO.UpdateResources."); + } return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new ConditionalUpdateFn(fhirStore, type))) + .apply(ParDo.of(new ConditionalUpdateFn(fhirStore, type, searchParametersFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** The type Write Fhir fn. */ + /** + * The type Conditional update fn. + */ static class ConditionalUpdateFn extends DoFn> { private Counter failedBundles = Metrics.counter(ConditionalUpdateFn.class, "failed-bundles"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); - /** The Fhir store. */ private final String fhirStore; private final String type; + private SerializableFunction> searchParametersFunction; /** - * Instantiates a new Write Fhir fn. - * @param fhirStore the Fhir store - * @param type + * Instantiates a new Conditional update fn. + * + * @param fhirStore the fhir store + * @param type the type + * @param searchParametersFunction the search parameters function */ - ConditionalUpdateFn(String fhirStore, String type) { + ConditionalUpdateFn(String fhirStore, String type, + SerializableFunction> searchParametersFunction) { this.fhirStore = fhirStore; this.type = type; + this.searchParametersFunction = searchParametersFunction; } - /** - * Initialize healthcare client. + * Init client. * * @throws IOException the io exception */ @@ -1389,7 +1533,7 @@ public void initClient() throws IOException { } /** - * ConditionalUpdate resources. + * Conditional update. * * @param context the context */ @@ -1399,7 +1543,8 @@ public void conditionalUpdate(ProcessContext context) { try { // Validate that data was set to valid JSON. mapper.readTree(body); - client.fhirConditionalUpdate(fhirStore, type, body); + Map searchParameters = searchParametersFunction.apply(body); + client.fhirConditionalUpdate(fhirStore, type, body, searchParameters); } catch (IOException | HealthcareHttpException e) { failedBundles.inc(); context.output(HealthcareIOError.of(body, e)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index efa594112ee4..6606f19facf9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -27,6 +27,7 @@ import com.google.api.services.healthcare.v1beta1.model.Operation; import java.io.IOException; import java.text.ParseException; +import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; @@ -109,7 +110,8 @@ HttpBody executeFhirBundle(String fhirStore, String bundle) HttpBody fhirCreate(String fhirStore, String type, String resource, @Nullable String ifNoneExist) throws IOException, HealthcareHttpException; - HttpBody fhirConditionalUpdate(String fhirStore, String relativeResourceName, String resource) + HttpBody fhirConditionalUpdate(String fhirStore, String relativeResourceName, String resource, + Map searchParameters) throws IOException, HealthcareHttpException; HttpBody fhirUpdate(String fhirStore, String type, String resource) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index a8582dd2971e..50c598858e33 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -60,6 +60,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; import org.apache.http.HttpEntity; import org.apache.http.HttpResponse; +import org.apache.http.NameValuePair; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpUriRequest; import org.apache.http.client.methods.RequestBuilder; @@ -68,6 +69,7 @@ import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.DefaultHttpRequestRetryHandler; import org.apache.http.impl.client.HttpClients; +import org.apache.http.message.BasicNameValuePair; import org.apache.http.util.EntityUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -299,6 +301,7 @@ static class FhirHttpRequest { private Method method; private String pathSuffix; private Map headers; + private Map parameters; enum Method { @@ -332,7 +335,10 @@ public FhirHttpRequest setMethod(Method method){ return this; } - + public FhirHttpRequest setParameters(Map parameters){ + this.parameters= parameters; + return this; + } } @@ -364,25 +370,32 @@ public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) throws I case PUT: requestBuilder = RequestBuilder.put(); break; - case POST: + case POST: // fallthrough default: requestBuilder = RequestBuilder.post(); } - HttpUriRequest request = requestBuilder - .setUri(uri) - .setEntity(requestEntity) - .addHeader("User-Agent", USER_AGENT) - .addHeader("Content-Type", FHIRSTORE_HEADER_CONTENT_TYPE) - .addHeader("Accept-Charset", FHIRSTORE_HEADER_ACCEPT_CHARSET) - .addHeader("Accept", FHIRSTORE_HEADER_ACCEPT) - .build(); + // add common headers + requestBuilder + .setUri(uri) + .setEntity(requestEntity) + .addHeader("User-Agent", USER_AGENT) + .addHeader("Content-Type", FHIRSTORE_HEADER_CONTENT_TYPE) + .addHeader("Accept-Charset", FHIRSTORE_HEADER_ACCEPT_CHARSET) + .addHeader("Accept", FHIRSTORE_HEADER_ACCEPT); + + // add additional headers + for (Map.Entry param: fhirHttpRequest.parameters.entrySet()){ + requestBuilder.addParameter(param.getKey(), param.getValue()); + } - // Add headers - for (Map.Entry header: fhirHttpRequest.headers.entrySet()){ - request.addHeader(header.getKey(), header.getValue()); + // add additional parameters + for (Map.Entry param: fhirHttpRequest.headers.entrySet()){ + requestBuilder.addHeader(param.getKey(), param.getValue()); } + HttpUriRequest request = requestBuilder.build(); + HttpResponse response = httpClient.execute(request); HttpEntity responseEntity = response.getEntity(); String content = EntityUtils.toString(responseEntity); @@ -417,11 +430,12 @@ public HttpBody fhirCreate(String fhirStore, String type, String resource, } @Override - public HttpBody fhirConditionalUpdate(String fhirStore, String type, String resource) + public HttpBody fhirConditionalUpdate(String fhirStore, String type, String resource, Map searchParameters) throws IOException, HealthcareHttpException { return executeFhirHttpRequest( FhirHttpRequest.of(fhirStore, resource) .setPathSuffix("/" + type) + .setParameters(searchParameters) .setMethod(Method.PUT)); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java index 24677ca3fb56..e8e06c60493c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java @@ -126,4 +126,6 @@ public void testFhirIO_Import() { pipeline.run().waitUntilFinish(); } + + // TODO(jaketf) add IT for conditional create, update, conditional update transforms. } From 0aa92fe0a808e0f6669725ece6db5bd07591c3cf Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 13 May 2020 14:49:05 -0700 Subject: [PATCH 009/151] remove references to _GroupByKeyOnly Change-Id: I0917d6495074204761d91579e5e5572401042a18 --- sdks/python/apache_beam/transforms/ptransform_test.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index ee007ff653f6..1cdd273fb3c4 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -52,7 +52,6 @@ from apache_beam.testing.util import equal_to from apache_beam.transforms import WindowInto from apache_beam.transforms import window -from apache_beam.transforms.core import _GroupByKeyOnly from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display import DisplayDataItem from apache_beam.transforms.ptransform import PTransform @@ -680,7 +679,7 @@ def test_group_by_key_only_input_must_be_kv_pairs(self): with self.assertRaises(typehints.TypeCheckError) as cm: with TestPipeline() as pipeline: pcolls = pipeline | 'A' >> beam.Create(['a', 'b', 'f']) - pcolls | 'D' >> _GroupByKeyOnly() + pcolls | 'D' >> beam.GroupByKey() expected_error_prefix = ( 'Input type hint violation at D: expected ' @@ -1233,7 +1232,7 @@ def test_group_by_key_only_output_type_deduction(self): | ( 'Pair' >> beam.Map(lambda x: (x, ord(x))).with_output_types( typing.Tuple[str, str])) - | _GroupByKeyOnly()) + | beam.GroupByKey()) # Output type should correctly be deduced. # GBK-only should deduce that Tuple[A, B] is turned into @@ -1261,7 +1260,7 @@ def test_group_by_key_only_does_not_type_check(self): ( self.p | beam.Create([1, 2, 3]).with_output_types(int) - | 'F' >> _GroupByKeyOnly()) + | 'F' >> beam.GroupByKey()) self.assertStartswith( e.exception.args[0], @@ -1309,7 +1308,7 @@ def test_pipeline_checking_gbk_insufficient_type_information(self): self.p | 'Nums' >> beam.Create(range(5)).with_output_types(int) | 'ModDup' >> beam.Map(lambda x: (x % 2, x)) - | _GroupByKeyOnly()) + | beam.GroupByKey()) self.assertEqual( 'Pipeline type checking is enabled, however no output ' @@ -2244,7 +2243,7 @@ def test_pardo_type_inference(self): def test_gbk_type_inference(self): self.assertEqual( typehints.Tuple[str, typehints.Iterable[int]], - _GroupByKeyOnly().infer_output_type(typehints.KV[str, int])) + beam.GroupByKey().infer_output_type(typehints.KV[str, int])) def test_pipeline_inference(self): created = self.p | beam.Create(['a', 'b', 'c']) From b1b612fdafd5a10aa417cd47421b405efc449f8a Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Wed, 13 May 2020 15:30:09 -0700 Subject: [PATCH 010/151] generify and document new interfaces for conditional create and conditional update --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 295 ++++++++++++------ 1 file changed, 198 insertions(+), 97 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 3ad15ee78515..e5efde50d587 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -48,6 +48,7 @@ import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.ExecuteBundles.ExecuteBundlesFn; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.metrics.Counter; @@ -121,9 +122,20 @@ * FhirIO.Write#fhirStoresImport(String, String, String, FhirIO.Import.ContentStructure)} This will * return a {@link FhirIO.Write.Result} on which you can call {@link * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link - * HealthcareIOError} containing the {@link String} that failed to be ingested and the exception.

    Example

    {@code
    + * HealthcareIOError} containing the {@link String} that failed to be ingested and the     exception.
    + *
    + * 

    Conditional Creating / Updating Resources

    + * {@link FhirIO} supports interfaces for conditional update. + * These can be useful to handle scenarios where an executeBundle failed. For example if you tried + * to create a resource that already exists you can grab the faield bodies of your + * {@link FhirIO.ExecuteBundles} transform with {@link Result#getFailedBodies()} perform some logic + * on the reason for failures and if appropriate route this to {@link FhirIO.ConditionalUpdate} or + * {@link FhirIO.CreateResources} to take the appropriate action on your FHIR store. + * + *

    Example

    {@code
      * Pipeline pipeline = ...
      *
    + * // [Begin Reading]
      * // Tail the FHIR store by retrieving resources based on Pub/Sub notifications.
      * FhirIO.Read.Result readResult = p
      *   .apply("Read FHIR notifications",
    @@ -133,8 +145,11 @@
      * // happily retrived messages
      * PCollection resources = readResult.getResources();
      * // message IDs that couldn't be retrieved + error context
    + *
      * PCollection> failedReads = readResult.getFailedReads();
    + * // [End Reading]
      *
    + * // [Beign Writing]
      * failedReads.apply("Write Message IDs / Stacktrace for Failed Reads to BigQuery",
      *     BigQueryIO
      *         .write()
    @@ -145,17 +160,53 @@
      * FhirIO.Write.Result writeResult =
      *     output.apply("Execute FHIR Bundles", FhirIO.executeBundles(options.getExistingFhirStore()));
      *
    + * // Alternatively you could use import for high throughput to a new store.
    + * FhirIO.Write.Result writeResult =
    + *     output.apply("Import FHIR Resources", FhirIO.executeBundles(options.getNewFhirStore()));
    + * // [End Writing ]
    + *
      * PCollection> failedBundles = writeResult.getFailedInsertsWithErr();
      *
    + * // [Begin Writing to Dead Letter Queue]
      * failedBundles.apply("Write failed bundles to BigQuery",
      *     BigQueryIO
      *         .write()
      *         .to(option.getBQFhirExecuteBundlesDeadLetterTable())
      *         .withFormatFunction(new HealthcareIOErrorToTableRow()));
    + * // [End Writing to Dead Letter Queue]
    + *
    + * // Alternatively you may want to handle DeadLetter with conditional update
    + * // [Begin Reconciliation with Conditional Update]
    + * failedBundles
    + *     .apply("Reconcile with Conditional Update",
    + *         FhirIO.ConditionalUpdate(fhirStore)
    + *             .withFormatBodyFunction(HealthcareIOError::getDataResource)
    + *             .withTypeFunction((HealthcareIOError err) -> {
    + *               String body = err.getDataResource();
    + *               // TODO(user) insert logic to exctract type.
    + *               return params;
    + *             })
    + *             .withSearchParametersFunction((HealthcareIOError err) -> {
    + *               String body = err.getDataResource();
    + *               Map params = new HashMap();
    + *               // TODO(user) insert logic to exctract search query parameters.
    + *               return params;
    + *             });
    + * // [End Reconciliation with Conditional Update]
    + *
    + * // Alternatively you may want to handle DeadLetter with conditional create
    + * // [Begin Reconciliation with Conditional Update]
    + * failedBundles
    + *     .apply("Reconcile with Conditional Create",
    + *         FhirIO.CreateResources(fhirStore)
    + *             .withFormatBodyFunction(HealthcareIOError::getDataResource)
    + *             .withIfNotExistsFunction((HealthcareIOError err) -> {
    + *               String body = err.getDataResource();
    + *               // TODO(user) insert logic to exctract a query to be used in If-Not-Exists header.
    + *               return params;
    + *             });
    + * // [End Reconciliation with Conditional Update]
      *
    - * // Alternatively you could use import for high throughput to a new store.
    - * FhirIO.Write.Result writeResult =
    - *     output.apply("Import FHIR Resources", FhirIO.executeBundles(options.getNewFhirStore()));
      * }*** 
    */ public class FhirIO { @@ -231,7 +282,7 @@ public static class Result implements POutput, PInput { PCollectionTuple pct; /** - * ConditionalUpdate FhirIO.Read.Result form PCollectionTuple with OUT and DEAD_LETTER tags. + * FhirIO.Read.Result form PCollectionTuple with OUT and DEAD_LETTER tags. * * @param pct the pct * @return the read result @@ -617,7 +668,7 @@ private static Write.Builder write(String fhirStore) { } /** - * ConditionalUpdate Method creates a single FHIR resource. @see * * @param fhirStore the hl 7 v 2 store @@ -745,7 +796,7 @@ public Result expand(PCollection input) { * Writes each bundle of elements to a new-line delimited JSON file on GCS and issues a * fhirStores.import Request for that file. This is intended for batch use only to facilitate * large backfills to empty FHIR stores and should not be used with unbounded PCollections. If - * your use case is streaming checkout using {@link ConditionalUpdate} to more safely execute + * your use case is streaming checkout using {@link Import} to more safely execute * bundles as transactions which is safer practice for a use on a "live" FHIR store. */ public static class Import extends Write { @@ -1239,12 +1290,15 @@ public void executeBundles(ProcessContext context) { /** * {@link PTransform} for Creating FHIR resources. + * * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/create */ - public static class CreateResources extends PTransform, Write.Result> { + public static class CreateResources extends PTransform, Write.Result> { private final String fhirStore; private final String type; - private SerializableFunction ifNoneExistFunction; + private SerializableFunction ifNoneExistFunction; + private SerializableFunction formatBodyFunction; + private static final Logger LOG = LoggerFactory.getLogger(CreateResources.class); /** * Instantiates a new Create resources transform. @@ -1279,33 +1333,57 @@ public static class CreateResources extends PTransform, Writ * @return the create resources */ public CreateResources withIfNotExistFunction( - SerializableFunction ifNoneExistFunction){ + SerializableFunction ifNoneExistFunction){ this.ifNoneExistFunction = ifNoneExistFunction; return this; } + /** + * With format body function create resources. + * + * @param formatBodyFunction the format body function + * @return the create resources + */ + public CreateResources withFormatBodyFunction( + SerializableFunction formatBodyFunction){ + this.formatBodyFunction= formatBodyFunction; + return this; + } + @Override - public FhirIO.Write.Result expand(PCollection input) { + public FhirIO.Write.Result expand(PCollection input) { + checkArgument( + formatBodyFunction != null, + "FhirIO.CreateResources should always be called with a " + + "withFromatBodyFunction"); + + if (ifNoneExistFunction == null){ + LOG.info("ifNoneExistFunction was null will create resources unconditionally"); + } + return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new CreateFn(fhirStore, type, ifNoneExistFunction))) + .apply(ParDo.of(new CreateFn(fhirStore, type, formatBodyFunction, ifNoneExistFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - static class CreateFn extends DoFn> { + static class CreateFn extends DoFn> { private Counter failedBundles = Metrics.counter(CreateFn.class, "failed-bundles"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); private final String fhirStore; private final String type; - private SerializableFunction ifNoneExistFunction; + private SerializableFunction ifNoneExistFunction; + private SerializableFunction formatBodyFunction; CreateFn(String fhirStore, String type, - SerializableFunction ifNoneExistFunction) { + SerializableFunction formatBodyFunction, + @Nullable SerializableFunction ifNoneExistFunction) { this.fhirStore = fhirStore; this.type = type; + this.formatBodyFunction= formatBodyFunction; this.ifNoneExistFunction = ifNoneExistFunction; } @@ -1316,14 +1394,15 @@ public void initClient() throws IOException { @ProcessElement public void create(ProcessContext context) { - String body = context.element(); + T input = context.element(); + String body = formatBodyFunction.apply(input); try { // Validate that data was set to valid JSON. - mapper.readTree(body); + mapper.readTree(input.toString()); if (ifNoneExistFunction != null){ - String ifNoneExistQuery = ifNoneExistFunction.apply(body); + String ifNoneExistQuery = ifNoneExistFunction.apply(input); client.fhirCreate(fhirStore, type, body, ifNoneExistQuery); - } else{ + } else { client.fhirCreate(fhirStore, type, body, null); } } catch (IOException | HealthcareHttpException e) { @@ -1339,10 +1418,12 @@ public void create(ProcessContext context) { * * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/update */ - public static class UpdateResources extends PTransform, Write.Result> { + public static class UpdateResources extends PTransform, Write.Result> { private final String fhirStore; - private SerializableFunction resourceNameFunction; + private SerializableFunction formatBodyFunction; + private SerializableFunction resourceNameFunction; public static final TupleTag UPDATE_RESULTS = new TupleTag() {}; + private static final Logger LOG = LoggerFactory.getLogger(UpdateResources.class); /** * Instantiates a new Update resources. @@ -1360,11 +1441,23 @@ public static class UpdateResources extends PTransform, Writ * @return the update resources */ public UpdateResources withResourceNameFunction( - SerializableFunction resourceNameFunction){ + SerializableFunction resourceNameFunction){ this.resourceNameFunction = resourceNameFunction; return this; } + /** + * With format body function update resources. + * + * @param formatBodyFunction the format body function + * @return the update resources + */ + public UpdateResources withFormatBodyFunction( + SerializableFunction formatBodyFunction){ + this.formatBodyFunction = formatBodyFunction; + return this; + } + /** * Instantiates a new Update resources. * @@ -1375,59 +1468,48 @@ public UpdateResources withResourceNameFunction( } @Override - public FhirIO.Write.Result expand(PCollection input) { + public FhirIO.Write.Result expand(PCollection input) { + checkArgument( + formatBodyFunction != null, + "FhirIO.UpdateResources should always be called with a " + + "withFormatBodyFunction."); checkArgument(resourceNameFunction == null, "resourceNameFunction must be set when using FhirIO.UpdateResources"); return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new UpdateFn(fhirStore, resourceNameFunction))) + .apply(ParDo.of(new UpdateFn(fhirStore, formatBodyFunction, resourceNameFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** - * The type Update fn. - */ - static class UpdateFn extends DoFn> { + static class UpdateFn extends DoFn> { private Counter failedUpdates = Metrics.counter(UpdateFn.class, "failed-updates"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); private final String fhirStore; - private SerializableFunction resourceNameFunction; + private SerializableFunction formatBodyFunction; + private SerializableFunction resourceNameFunction; - /** - * Instantiates a new Update fn. - * - * @param fhirStore the fhir store - */ - UpdateFn(String fhirStore, SerializableFunction resourceNameFunction) { + UpdateFn(String fhirStore, SerializableFunction formatBodyFunction, SerializableFunction resourceNameFunction) { this.fhirStore = fhirStore; + this.formatBodyFunction = formatBodyFunction; this.resourceNameFunction = resourceNameFunction; } - /** - * Init client. - * - * @throws IOException the io exception - */ @Setup public void initClient() throws IOException { this.client = new HttpHealthcareApiClient(); } - /** - * Update. - * - * @param context the context - */ @ProcessElement public void update(ProcessContext context) { - String body = context.element(); + T input = context.element(); + String body = formatBodyFunction.apply(input); try { // Validate that data was set to valid JSON. mapper.readTree(body); - String resourceName = resourceNameFunction.apply(body); + String resourceName = resourceNameFunction.apply(input); HttpBody result = client.fhirUpdate(fhirStore, resourceName, body); context.output(UPDATE_RESULTS, result.getData()); } catch (IOException | HealthcareHttpException e) { @@ -1439,23 +1521,23 @@ public void update(ProcessContext context) { } /** - * The type Conditional update. + * {@link PTransform} to perform Conditional updates on the FHIR store. + * + * https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/conditionalUpdate */ - public static class ConditionalUpdate extends PTransform, Write.Result> { + public static class ConditionalUpdate extends PTransform, Write.Result> { private final String fhirStore; - private final String type; - private SerializableFunction searchParametersFunction; + private SerializableFunction> searchParametersFunction; + private SerializableFunction typeFunction; + private SerializableFunction formatBodyFunction; /** * Instantiates a new Conditional update. * * @param fhirStore the fhir store - * @param type the type - * @param conditional the conditional */ - ConditionalUpdate(ValueProvider fhirStore, String type, boolean conditional) { + ConditionalUpdate(ValueProvider fhirStore) { this.fhirStore = fhirStore.get(); - this.type = type; } /** @@ -1465,88 +1547,107 @@ public static class ConditionalUpdate extends PTransform, Wr * @return the conditional update */ public ConditionalUpdate withSearchParametersFunction( - SerializableFunction> searchParametersFunction){ + SerializableFunction> searchParametersFunction){ this.searchParametersFunction = searchParametersFunction; return this; } + /** + * With type function conditional update. + * + * @param typeFunction the type function + * @return the conditional update + */ + public ConditionalUpdate withTypeFunction( + SerializableFunction typeFunction){ + this.typeFunction = typeFunction; + return this; + } + + /** + * With format body function conditional update. + * + * @param formatBodyFunction the format body function + * @return the conditional update + */ + public ConditionalUpdate withFormatBodyFunction( + SerializableFunction formatBodyFunction){ + this.formatBodyFunction = formatBodyFunction; + return this; + } + /** * Instantiates a new Conditional update. * * @param fhirStore the fhir store - * @param type the type - * @param conditional the conditional */ - ConditionalUpdate(String fhirStore, String type, boolean conditional) { + ConditionalUpdate(String fhirStore) { this.fhirStore = fhirStore; - this.type = type; } + @Override - public FhirIO.Write.Result expand(PCollection input) { - if (searchParametersFunction == null){ - throw new IllegalArgumentException( - "FhirIO.ConditionalUpdate should always be called with a searchParametersFunction." - + "If this does not meet your use case consider usiing FhirIO.UpdateResources."); - } + public FhirIO.Write.Result expand(PCollection input) { + checkArgument( + typeFunction != null, + "FhirIO.ConditionalUpdate should always be called with a " + + "withTypeFunction."); + checkArgument( + formatBodyFunction != null, + "FhirIO.ConditionalUpdate should always be called with a " + + "withFormatBodyFunction."); + checkArgument( + searchParametersFunction != null, + "FhirIO.ConditionalUpdate should always be called with a " + + "withSearchParametersFunction. If this does not meet your use case consider usiing " + + "FhirIO.UpdateResources."); + + return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new ConditionalUpdateFn(fhirStore, type, searchParametersFunction))) + .apply(ParDo.of(new ConditionalUpdateFn( + fhirStore, typeFunction, searchParametersFunction, formatBodyFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } - /** - * The type Conditional update fn. - */ - static class ConditionalUpdateFn extends DoFn> { + static class ConditionalUpdateFn extends DoFn> { - private Counter failedBundles = Metrics.counter(ConditionalUpdateFn.class, "failed-bundles"); + private Counter failedConditionalUpdates = Metrics.counter(ConditionalUpdateFn.class, + "failed-conditional-updates"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); private final String fhirStore; - private final String type; - private SerializableFunction> searchParametersFunction; + private SerializableFunction> searchParametersFunction; + private SerializableFunction typeFunction; + private SerializableFunction formatBodyFunction; - /** - * Instantiates a new Conditional update fn. - * - * @param fhirStore the fhir store - * @param type the type - * @param searchParametersFunction the search parameters function - */ - ConditionalUpdateFn(String fhirStore, String type, - SerializableFunction> searchParametersFunction) { + ConditionalUpdateFn(String fhirStore, SerializableFunction typeFunction, + SerializableFunction> searchParametersFunction, + SerializableFunction formatBodyFunction) { this.fhirStore = fhirStore; - this.type = type; + this.typeFunction = typeFunction; this.searchParametersFunction = searchParametersFunction; + this.formatBodyFunction = formatBodyFunction; } - /** - * Init client. - * - * @throws IOException the io exception - */ @Setup public void initClient() throws IOException { this.client = new HttpHealthcareApiClient(); } - /** - * Conditional update. - * - * @param context the context - */ @ProcessElement public void conditionalUpdate(ProcessContext context) { - String body = context.element(); + T input = context.element(); + String type = typeFunction.apply(input); + String body = formatBodyFunction.apply(input); try { // Validate that data was set to valid JSON. mapper.readTree(body); - Map searchParameters = searchParametersFunction.apply(body); + Map searchParameters = searchParametersFunction.apply(input); client.fhirConditionalUpdate(fhirStore, type, body, searchParameters); } catch (IOException | HealthcareHttpException e) { - failedBundles.inc(); + failedConditionalUpdates.inc(); context.output(HealthcareIOError.of(body, e)); } } From cc405fdc375de0b73797c06f16522f374d511909 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 13 May 2020 16:25:25 -0700 Subject: [PATCH 011/151] add back the ReifyWindows to the GBK, some internal tests depend on it. Change-Id: Ibce789eaa56ca93842d94cd23bd6d4aadb05e08d --- .../runners/direct/direct_runner.py | 26 +++---------------- sdks/python/apache_beam/transforms/core.py | 17 ++++++++++++ 2 files changed, 21 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 8fa6c902900d..51b064a88114 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -51,9 +51,7 @@ from apache_beam.transforms.core import DoFn from apache_beam.transforms.core import ParDo from apache_beam.transforms.ptransform import PTransform -from apache_beam.transforms.window import WindowedValue from apache_beam.typehints import trivial_inference -from apache_beam.typehints.decorators import TypeCheckError # Note that the BundleBasedDirectRunner and SwitchingDirectRunner names are # experimental and have no backwards compatibility guarantees. @@ -227,23 +225,6 @@ def from_runner_api_parameter(unused_ptransform, payload, context): @typehints.with_output_types(typing.Tuple[K, typing.Iterable[V]]) class _GroupByKey(PTransform): """The DirectRunner GroupByKey implementation.""" - class ReifyWindows(DoFn): - def process( - self, element, window=DoFn.WindowParam, timestamp=DoFn.TimestampParam): - try: - k, v = element - except TypeError: - raise TypeCheckError( - 'Input to GroupByKey must be a PCollection with ' - 'elements compatible with KV[A, B]') - - return [(k, WindowedValue(v, timestamp, [window]))] - - def infer_output_type(self, input_type): - key_type, value_type = trivial_inference.key_value_types(input_type) - return typehints.Iterable[typehints.KV[ - key_type, typehints.WindowedValue[value_type]]] # type: ignore[misc] - def expand(self, pcoll): # Imported here to avoid circular dependencies. # pylint: disable=wrong-import-order, wrong-import-position @@ -274,8 +255,9 @@ def expand(self, pcoll): # pylint: disable=bad-continuation return ( pcoll - | 'ReifyWindows' >> - (ParDo(self.ReifyWindows()).with_output_types(reify_output_type)) + | 'ReifyWindows' >> ( + ParDo(beam.GroupByKey.ReifyWindows()).with_output_types( + reify_output_type)) | 'GroupByKey' >> ( _GroupByKeyOnly().with_input_types( reify_output_type).with_output_types(gbk_input_type)) @@ -287,7 +269,7 @@ def expand(self, pcoll): # The input_type is None, run the default return ( pcoll - | 'ReifyWindows' >> ParDo(self.ReifyWindows()) + | 'ReifyWindows' >> ParDo(beam.GroupByKey.ReifyWindows()) | 'GroupByKey' >> _GroupByKeyOnly() | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 665d7ec4ebea..fe87117d1e43 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2208,6 +2208,23 @@ class GroupByKey(PTransform): The implementation here is used only when run on the local direct runner. """ + class ReifyWindows(DoFn): + def process( + self, element, window=DoFn.WindowParam, timestamp=DoFn.TimestampParam): + try: + k, v = element + except TypeError: + raise TypeCheckError( + 'Input to GroupByKey must be a PCollection with ' + 'elements compatible with KV[A, B]') + + return [(k, WindowedValue(v, timestamp, [window]))] + + def infer_output_type(self, input_type): + key_type, value_type = trivial_inference.key_value_types(input_type) + return typehints.Iterable[typehints.KV[ + key_type, typehints.WindowedValue[value_type]]] # type: ignore[misc] + def expand(self, pcoll): return pvalue.PCollection.from_(pcoll) From 9be5a29ba3ffc632b049fd18299be40f19ab74aa Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Fri, 15 May 2020 11:26:25 -0700 Subject: [PATCH 012/151] Updated Videos and Podcasts page --- .../resources/videos-and-podcasts.md | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/resources/videos-and-podcasts.md b/website/www/site/content/en/documentation/resources/videos-and-podcasts.md index b1d0a0bb6371..651cd8856cd8 100644 --- a/website/www/site/content/en/documentation/resources/videos-and-podcasts.md +++ b/website/www/site/content/en/documentation/resources/videos-and-podcasts.md @@ -65,6 +65,24 @@ Presented by Frances Perry, *Apache Beam PPMC member* The following resources present Apache Beam partnerships. +### Distributed Processing for Machine Learning Production Pipelines + +Flink Forward, 2020 + +Presented by Ahmet Altay, Robert Crowe, Reza Rokni + + +
    + +### TensorFlow Extended: An End-to-End Machine Learning Platform for TensorFlow + +Spark+AI, San Francisco, 2019 + +Presented by Konstantinos Katsiapis, Ahmet Altay + + +
    + ### Flink and Beam: Current State & Roadmap Flink Forward, Berlin, 2016 @@ -129,4 +147,4 @@ Your browser does not support the audio element. ## Next Steps -* Take a self-paced tour through our [Learning Resources](/documentation/resources/learning-resources). \ No newline at end of file +* Take a self-paced tour through our [Learning Resources](/documentation/resources/learning-resources). From 301c32b0d45377172c571e3a01dba4bfcd4f467a Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Fri, 15 May 2020 17:38:42 -0400 Subject: [PATCH 013/151] Update Beam website to release 2.21.0. --- website/www/site/config.toml | 2 +- website/www/site/content/en/get-started/downloads.md | 7 +++++++ website/www/site/static/.htaccess | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/website/www/site/config.toml b/website/www/site/config.toml index 9c4bcd179672..5211e364a950 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.20.0" +release_latest = "2.21.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 76a032c89369..14ae4d683162 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -87,6 +87,13 @@ versions denoted `0.x.y`. ## Releases +### 2.21.0 (2020-05-xx) +Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.21.0/apache-beam-2.21.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.21.0/apache-beam-2.21.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.21.0/apache-beam-2.21.0-source-release.zip.asc). + +[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347143). + ### 2.20.0 (2020-04-15) Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.20.0/apache-beam-2.20.0-source-release.zip). [SHA-512](https://downloads.apache.org/beam/2.20.0/apache-beam-2.20.0-source-release.zip.sha512). diff --git a/website/www/site/static/.htaccess b/website/www/site/static/.htaccess index 2ccc960afe4e..916c7558eb66 100644 --- a/website/www/site/static/.htaccess +++ b/website/www/site/static/.htaccess @@ -21,4 +21,4 @@ RewriteRule ^(.*)$ https://beam.apache.org/$1 [L,R=301] # The following redirect maintains the previously supported URLs. RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam.apache.org/releases/$1$2" # Keep this updated to point to the current release. -RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.20.0$2" +RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.21.0$2" From ad15754e746c10e615ba88f5d00f79a2dca09894 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Fri, 15 May 2020 19:53:41 -0400 Subject: [PATCH 014/151] Add blog post announcing the Beam 2.21.0 release. --- CHANGES.md | 3 +- .../www/site/content/en/blog/beam-2.21.0.md | 97 +++++++++++++++++++ 2 files changed, 99 insertions(+), 1 deletion(-) create mode 100644 website/www/site/content/en/blog/beam-2.21.0.md diff --git a/CHANGES.md b/CHANGES.md index 29db42c152ce..c55967a18bfb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -75,7 +75,7 @@ * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). -# [2.21.0] - Unreleased (In Progress) +# [2.21.0] - 2020-05-xx ## Highlights @@ -139,6 +139,7 @@ conversion to beam schema options. *Remark: Schema aware is still experimental.* ## Deprecations * Java SDK: Beam Schema FieldType.getMetadata is now deprecated and is replaced by the Beam Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://issues.apache.org/jira/browse/BEAM-9704)) +* The `--zone` option in the Dataflow runner is now deprecated. Please use `--worker_zone` instead. ([BEAM-9716](https://issues.apache.org/jira/browse/BEAM-9716)) ## Known Issues diff --git a/website/www/site/content/en/blog/beam-2.21.0.md b/website/www/site/content/en/blog/beam-2.21.0.md new file mode 100644 index 000000000000..286d71b6a20b --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.21.0.md @@ -0,0 +1,97 @@ +--- +title: "Apache Beam 2.21.0" +date: 2020-05-xx 00:00:01 -0800 +categories: + - blog +authors: + - ibzib +--- + + +We are happy to present the new 2.21.0 release of Beam. This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#xxxx-xxxx) for this release. +For more information on changes in 2.21.0, check out the +[detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347143). + +## I/Os +* Python: Deprecated module `apache_beam.io.gcp.datastore.v1` has been removed +as the client it uses is out of date and does not support Python 3 +([BEAM-9529](https://issues.apache.org/jira/browse/BEAM-9529)). +Please migrate your code to use +[apache_beam.io.gcp.datastore.**v1new**](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.datastore.v1new.datastoreio.html). +See the updated +[datastore_wordcount](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py) +for example usage. +* Python SDK: Added integration tests and updated batch write functionality for Google Cloud Spanner transform ([BEAM-8949](https://issues.apache.org/jira/browse/BEAM-8949)). + +## New Features / Improvements +* Python SDK will now use Python 3 type annotations as pipeline type hints. +([#10717](https://github.com/apache/beam/pull/10717)) + + If you suspect that this feature is causing your pipeline to fail, calling + `apache_beam.typehints.disable_type_annotations()` before pipeline creation + will disable is completely, and decorating specific functions (such as + `process()`) with `@apache_beam.typehints.no_annotations` will disable it + for that function. + + More details will be in + [Ensuring Python Type Safety](https://beam.apache.org/documentation/sdks/python-type-safety/) + and an upcoming + [blog post](https://beam.apache.org/blog/python/typing/2020/03/06/python-typing.html). + +* Java SDK: Introducing the concept of options in Beam Schema’s. These options add extra +context to fields and schemas. This replaces the current Beam metadata that is present +in a FieldType only, options are available in fields and row schemas. Schema options are +fully typed and can contain complex rows. *Remark: Schema aware is still experimental.* +([BEAM-9035](https://issues.apache.org/jira/browse/BEAM-9035)) +* Java SDK: The protobuf extension is fully schema aware and also includes protobuf option +conversion to beam schema options. *Remark: Schema aware is still experimental.* +([BEAM-9044](https://issues.apache.org/jira/browse/BEAM-9044)) +* Added ability to write to BigQuery via Avro file loads (Python) ([BEAM-8841](https://issues.apache.org/jira/browse/BEAM-8841)) + + By default, file loads will be done using JSON, but it is possible to + specify the temp_file_format parameter to perform file exports with AVRO. + AVRO-based file loads work by exporting Python types into Avro types, so + to switch to Avro-based loads, you will need to change your data types + from Json-compatible types (string-type dates and timestamp, long numeric + values as strings) into Python native types that are written to Avro + (Python's date, datetime types, decimal, etc). For more information + see https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-avro#avro_conversions. +* Added integration of Java SDK with Google Cloud AI VideoIntelligence service +([BEAM-9147](https://issues.apache.org/jira/browse/BEAM-9147)) +* Added integration of Java SDK with Google Cloud AI natural language processing API +([BEAM-9634](https://issues.apache.org/jira/browse/BEAM-9634)) +* `docker-pull-licenses` tag was introduced. Licenses/notices of third party dependencies will be added to the docker images when `docker-pull-licenses` was set. + The files are added to `/opt/apache/beam/third_party_licenses/`. + By default, no licenses/notices are added to the docker images. ([BEAM-9136](https://issues.apache.org/jira/browse/BEAM-9136)) + + +## Breaking Changes + +* Dataflow runner now requires the `--region` option to be set, unless a default value is set in the environment ([BEAM-9199](https://issues.apache.org/jira/browse/BEAM-9199)). See [here](https://cloud.google.com/dataflow/docs/concepts/regional-endpoints) for more details. +* HBaseIO.ReadAll now requires a PCollection of HBaseIO.Read objects instead of HBaseQuery objects ([BEAM-9279](https://issues.apache.org/jira/browse/BEAM-9279)). +* ProcessContext.updateWatermark has been removed in favor of using a WatermarkEstimator ([BEAM-9430](https://issues.apache.org/jira/browse/BEAM-9430)). +* Coder inference for PCollection of Row objects has been disabled ([BEAM-9569](https://issues.apache.org/jira/browse/BEAM-9569)). +* Go SDK docker images are no longer released until further notice. + +## Deprecations +* Java SDK: Beam Schema FieldType.getMetadata is now deprecated and is replaced by the Beam +Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://issues.apache.org/jira/browse/BEAM-9704)) +* The `--zone` option in the Dataflow runner is now deprecated. Please use `--worker_zone` instead. ([BEAM-9716](https://issues.apache.org/jira/browse/BEAM-9716)) + + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.21.0 release. Thank you to all contributors! + +Aaron Meihm, Adrian Eka, Ahmet Altay, AldairCoronel, Alex Van Boxel, Alexey Romanenko, Andrew Crites, Andrew Pilloud, Ankur Goenka, Badrul (Taki) Chowdhury, Bartok Jozsef, Boyuan Zhang, Brian Hulette, brucearctor, bumblebee-coming, Chad Dombrova, Chamikara Jayalath, Chie Hayashida, Chris Gorgolewski, Chuck Yang, Colm O hEigeartaigh, Curtis "Fjord" Hawthorne, Daniel Mills, Daniel Oliveira, David Yan, Elias Djurfeldt, Emiliano Capoccia, Etienne Chauchot, Fernando Diaz, Filipe Regadas, Gleb Kanterov, Hai Lu, Hannah Jiang, Harch Vardhan, Heejong Lee, Henry Suryawirawan, Hk-tang, Ismaël Mejía, Jacoby, Jan Lukavský, Jeroen Van Goey, jfarr, Jozef Vilcek, Kai Jiang, Kamil Wasilewski, Kenneth Knowles, KevinGG, Kyle Weaver, Kyoungha Min, Luke Cwik, Maximilian Michels, Michal Walenia, Ning Kang, Pablo Estrada, paul fisher, Piotr Szuberski, Reuven Lax, Robert Bradshaw, Robert Burke, Rose Nguyen, Rui Wang, Sam Rohde, Sam Whittle, Spoorti Kundargi, Steve Koonce, sunjincheng121, Ted Yun, Tesio, Thomas Weise, Tomo Suzuki, Udi Meiri, Valentyn Tymofieiev, Vasu Nori, Yichi Zhang, yoshiki.obata, Yueyang Qiu From eea7d10a5fcdaf700ab3f9593cca78e65799166b Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Mon, 18 May 2020 12:20:37 -0700 Subject: [PATCH 015/151] move bundle test resources --- ...Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json | 0 .../Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json | 0 ...nesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json | 0 .../Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json | 0 .../Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json | 0 .../DSTU2/hospitalInformation1586309771387.json | 0 .../DSTU2/practitionerInformation1586309771387.json | 0 ...erson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json | 0 ...Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json | 0 .../Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json | 0 ...Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json | 0 ...ymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json | 0 .../R4/hospitalInformation1586368892823.json | 0 .../R4/practitionerInformation1586368892823.json | 0 ...ander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json | 0 ...l991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json | 0 .../Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json | 0 .../Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json | 0 ...annette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json | 0 ...Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json | 0 .../STU3/hospitalInformation1586298239556.json | 0 .../STU3/practitionerInformation1586298239556.json | 0 22 files changed, 0 insertions(+), 0 deletions(-) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/hospitalInformation1586309771387.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/DSTU2/practitionerInformation1586309771387.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/hospitalInformation1586368892823.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/R4/practitionerInformation1586368892823.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/hospitalInformation1586298239556.json (100%) rename sdks/java/io/google-cloud-platform/src/test/resources/{ => transactional_bundles}/STU3/practitionerInformation1586298239556.json (100%) diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/hospitalInformation1586309771387.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/hospitalInformation1586309771387.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/practitionerInformation1586309771387.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/DSTU2/practitionerInformation1586309771387.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/hospitalInformation1586368892823.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/hospitalInformation1586368892823.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/practitionerInformation1586368892823.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/R4/practitionerInformation1586368892823.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/hospitalInformation1586298239556.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/hospitalInformation1586298239556.json diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/practitionerInformation1586298239556.json similarity index 100% rename from sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json rename to sdks/java/io/google-cloud-platform/src/test/resources/transactional_bundles/STU3/practitionerInformation1586298239556.json From 6cf105ae5823e01b793fb4ddbb346342cd606634 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Mon, 18 May 2020 12:40:43 -0700 Subject: [PATCH 016/151] Add to/from_runner_api_parameters to WriteToBigQuery Change-Id: Ifd969174d5a7744766173f35fd6b65a72ebfd924 --- sdks/python/apache_beam/io/gcp/bigquery.py | 72 +++++++++++++++++++ .../apache_beam/io/gcp/bigquery_test.py | 67 +++++++++++++++++ 2 files changed, 139 insertions(+) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index fd6b37087484..f0ad45715c49 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -271,6 +271,8 @@ def compute_table_name(row): from apache_beam.transforms import ParDo from apache_beam.transforms import PTransform from apache_beam.transforms.display import DisplayDataItem +from apache_beam.transforms.sideinputs import SIDE_INPUT_PREFIX +from apache_beam.transforms.sideinputs import get_sideinput_index from apache_beam.transforms.window import GlobalWindows from apache_beam.utils import retry from apache_beam.utils.annotations import deprecated @@ -1390,6 +1392,9 @@ def __init__( and https://cloud.google.com/bigquery/docs/loading-data-cloud-storage-json. """ + self._table = table + self._dataset = dataset + self._project = project self.table_reference = bigquery_tools.parse_table_reference( table, dataset, project) self.create_disposition = BigQueryDisposition.validate_create( @@ -1517,6 +1522,73 @@ def display_data(self): res['table'] = DisplayDataItem(tableSpec, label='Table') return res + def to_runner_api_parameter(self, context): + from apache_beam.internal import pickler + + # It'd be nice to name these according to their actual + # names/positions in the orignal argument list, but such a + # transformation is currently irreversible given how + # remove_objects_from_args and insert_values_in_args + # are currently implemented. + def serialize(side_inputs): + return {(SIDE_INPUT_PREFIX + '%s') % ix: + si.to_runner_api(context).SerializeToString() + for ix, + si in enumerate(side_inputs)} + + table_side_inputs = serialize(self.table_side_inputs) + schema_side_inputs = serialize(self.schema_side_inputs) + + config = { + 'table': self._table, + 'dataset': self._dataset, + 'project': self._project, + 'schema': self.schema, + 'create_disposition': self.create_disposition, + 'write_disposition': self.write_disposition, + 'kms_key': self.kms_key, + 'batch_size': self.batch_size, + 'max_file_size': self.max_file_size, + 'max_files_per_bundle': self.max_files_per_bundle, + 'custom_gcs_temp_location': self.custom_gcs_temp_location, + 'method': self.method, + 'insert_retry_strategy': self.insert_retry_strategy, + 'additional_bq_parameters': self.additional_bq_parameters, + 'table_side_inputs': table_side_inputs, + 'schema_side_inputs': schema_side_inputs, + 'triggering_frequency': self.triggering_frequency, + 'validate': self._validate, + 'temp_file_format': self._temp_file_format, + } + return 'beam:transform:write_to_big_query:v0', pickler.dumps(config) + + @PTransform.register_urn('beam:transform:write_to_big_query:v0', bytes) + def from_runner_api(unused_ptransform, payload, context): + from apache_beam.internal import pickler + from apache_beam.portability.api.beam_runner_api_pb2 import SideInput + + config = pickler.loads(payload) + + def deserialize(side_inputs): + deserialized_side_inputs = {} + for k, v in side_inputs.items(): + side_input = SideInput() + side_input.ParseFromString(v) + deserialized_side_inputs[k] = side_input + + # This is an ordered list stored as a dict (see the comments in + # to_runner_api_parameter above). + indexed_side_inputs = [( + get_sideinput_index(tag), + pvalue.AsSideInput.from_runner_api(si, context)) for tag, + si in deserialized_side_inputs.items()] + return [si for _, si in sorted(indexed_side_inputs)] + + config['table_side_inputs'] = deserialize(config['table_side_inputs']) + config['schema_side_inputs'] = deserialize(config['schema_side_inputs']) + + return WriteToBigQuery(**config) + class _PassThroughThenCleanup(PTransform): """A PTransform that invokes a DoFn after the input PCollection has been diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 8c2bfe8f0d76..5c0597869517 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -584,6 +584,73 @@ def test_schema_autodetect_not_allowed_with_avro_file_loads(self): schema=beam.io.gcp.bigquery.SCHEMA_AUTODETECT, temp_file_format=bigquery_tools.FileFormat.AVRO)) + def test_to_from_runner_api(self): + """Tests that serialization of WriteToBigQuery is correct. + + This is not intended to be a change-detector test. As such, this only tests + the more complicated serialization logic of parameters: ValueProviders, + callables, and side inputs. + """ + FULL_OUTPUT_TABLE = 'test_project:output_table' + + p = TestPipeline( + additional_pipeline_args=["--experiments=use_beam_bq_sink"]) + + # Used for testing side input parameters. + table_record_pcv = beam.pvalue.AsDict( + p | "MakeTable" >> beam.Create([('table', FULL_OUTPUT_TABLE)])) + + # Used for testing value provider parameters. + schema = value_provider.StaticValueProvider(str, '"a:str"') + + original = WriteToBigQuery( + table=lambda _, + side_input: side_input['table'], + table_side_inputs=(table_record_pcv, ), + schema=schema) + + # pylint: disable=expression-not-assigned + p | 'MyWriteToBigQuery' >> original + + # Run the pipeline through to generate a pipeline proto from an empty + # context. This ensures that the serialization code ran. + pipeline_proto, context = TestPipeline.from_runner_api( + p.to_runner_api(), p.runner, p.get_pipeline_options()).to_runner_api( + return_context=True) + + # Find the transform from the context. + write_to_bq_id = [ + k for k, + v in pipeline_proto.components.transforms.items() + if v.unique_name == 'MyWriteToBigQuery' + ][0] + deserialized_node = context.transforms.get_by_id(write_to_bq_id) + deserialized = deserialized_node.transform + self.assertIsInstance(deserialized, WriteToBigQuery) + + # Test that the serialization of a value provider is correct. + self.assertEqual(original.schema, deserialized.schema) + + # Test that the serialization of a callable is correct. + self.assertEqual( + deserialized._table(None, {'table': FULL_OUTPUT_TABLE}), + FULL_OUTPUT_TABLE) + + # Test that the serialization of a side input is correct. + self.assertEqual( + len(original.table_side_inputs), len(deserialized.table_side_inputs)) + original_side_input_data = original.table_side_inputs[0]._side_input_data() + deserialized_side_input_data = deserialized.table_side_inputs[ + 0]._side_input_data() + self.assertEqual( + original_side_input_data.access_pattern, + deserialized_side_input_data.access_pattern) + self.assertEqual( + original_side_input_data.window_mapping_fn, + deserialized_side_input_data.window_mapping_fn) + self.assertEqual( + original_side_input_data.view_fn, deserialized_side_input_data.view_fn) + @unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') class BigQueryStreamingInsertTransformTests(unittest.TestCase): From e80ee15f8f76f126d83a9a20ed837c985c09c934 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Mon, 18 May 2020 15:20:10 -0700 Subject: [PATCH 017/151] wip --- .../sdk/transforms/SerializableFunction.java | 3 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 101 ++++- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 5 +- .../sdk/io/gcp/healthcare/FhirIOWriteIT.java | 65 ++++ .../resources/resources/DSTU2/CarePlan.ndjson | 3 + .../resources/resources/DSTU2/Claim.ndjson | 25 ++ .../resources/DSTU2/Condition.ndjson | 4 + .../resources/DSTU2/DiagnosticReport.ndjson | 1 + .../resources/DSTU2/Encounter.ndjson | 16 + .../resources/DSTU2/ImagingStudy.ndjson | 1 + .../resources/DSTU2/Immunization.ndjson | 9 + .../resources/DSTU2/MedicationOrder.ndjson | 9 + .../resources/DSTU2/Observation.ndjson | 54 +++ .../resources/DSTU2/Organization.ndjson | 2 + .../resources/resources/DSTU2/Patient.ndjson | 1 + .../resources/DSTU2/Practitioner.ndjson | 2 + .../resources/DSTU2/Procedure.ndjson | 7 + .../hospitalInformation1589831190233.json | 183 +++++++++ .../practitionerInformation1589831190233.json | 135 +++++++ .../resources/resources/R4/CarePlan.ndjson | 3 + .../resources/resources/R4/CareTeam.ndjson | 3 + .../test/resources/resources/R4/Claim.ndjson | 25 ++ .../resources/resources/R4/Condition.ndjson | 4 + .../resources/R4/DiagnosticReport.ndjson | 17 + .../resources/R4/DocumentReference.ndjson | 16 + .../resources/resources/R4/Encounter.ndjson | 16 + .../resources/R4/ExplanationOfBenefit.ndjson | 16 + .../resources/R4/ImagingStudy.ndjson | 1 + .../resources/R4/Immunization.ndjson | 9 + .../resources/resources/R4/Location.ndjson | 2 + .../resources/R4/MedicationRequest.ndjson | 9 + .../resources/resources/R4/Observation.ndjson | 54 +++ .../resources/R4/Organization.ndjson | 2 + .../resources/resources/R4/Patient.ndjson | 1 + .../resources/R4/Practitioner.ndjson | 2 + .../resources/R4/PractitionerRole.ndjson | 2 + .../resources/resources/R4/Procedure.ndjson | 7 + .../resources/resources/R4/Provenance.ndjson | 1 + .../R4/hospitalInformation1589831190233.json | 318 ++++++++++++++++ .../practitionerInformation1589831190233.json | 354 ++++++++++++++++++ .../resources/resources/STU3/CarePlan.ndjson | 3 + .../resources/resources/STU3/Claim.ndjson | 25 ++ .../resources/resources/STU3/Condition.ndjson | 4 + .../resources/STU3/DiagnosticReport.ndjson | 1 + .../resources/resources/STU3/Encounter.ndjson | 16 + .../STU3/ExplanationOfBenefit.ndjson | 16 + .../resources/STU3/ImagingStudy.ndjson | 1 + .../resources/STU3/Immunization.ndjson | 9 + .../resources/STU3/MedicationRequest.ndjson | 9 + .../resources/STU3/Observation.ndjson | 54 +++ .../resources/STU3/Organization.ndjson | 2 + .../resources/resources/STU3/Patient.ndjson | 1 + .../resources/STU3/Practitioner.ndjson | 2 + .../resources/resources/STU3/Procedure.ndjson | 7 + .../hospitalInformation1589831190233.json | 234 ++++++++++++ .../practitionerInformation1589831190233.json | 135 +++++++ 56 files changed, 1994 insertions(+), 13 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/CarePlan.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Claim.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Condition.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/DiagnosticReport.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Encounter.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/ImagingStudy.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Immunization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/MedicationOrder.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Observation.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Organization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Patient.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Practitioner.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Procedure.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CarePlan.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CareTeam.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Claim.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Condition.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DiagnosticReport.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DocumentReference.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Encounter.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ExplanationOfBenefit.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ImagingStudy.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Immunization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Location.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/MedicationRequest.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Observation.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Organization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Patient.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Practitioner.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/PractitionerRole.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Procedure.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Provenance.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/CarePlan.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Claim.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Condition.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/DiagnosticReport.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Encounter.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ExplanationOfBenefit.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ImagingStudy.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Immunization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/MedicationRequest.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Observation.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Organization.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Patient.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Practitioner.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Procedure.ndjson create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java index a1dba9e688fe..3dbe893eef4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import com.fasterxml.jackson.core.JsonProcessingException; import java.io.Serializable; /** @@ -35,5 +36,5 @@ public interface SerializableFunction extends ProcessFunction, Serializable { /** Returns the result of invoking this function on the given input. */ @Override - OutputT apply(InputT input); + OutputT apply(InputT input) throws JsonProcessingException; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index e7edee513735..5446c58dc40c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -1250,6 +1250,27 @@ public void executeBundles(ProcessContext context) { } } + /** + * Create resources fhir io . create resources. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the fhir io . create resources + */ + public static FhirIO.CreateResources createResources(ValueProvider fhirStore){ + return new CreateResources(fhirStore); + } + + /** + * Create resources fhir io . create resources. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the fhir io . create resources + */ + public static FhirIO.CreateResources createResources(String fhirStore){ + return new CreateResources(fhirStore); + } /** * {@link PTransform} for Creating FHIR resources. * @@ -1257,31 +1278,27 @@ public void executeBundles(ProcessContext context) { */ public static class CreateResources extends PTransform, Write.Result> { private final String fhirStore; - private final String type; private SerializableFunction ifNoneExistFunction; private SerializableFunction formatBodyFunction; + private SerializableFunction typeFunction; private static final Logger LOG = LoggerFactory.getLogger(CreateResources.class); /** * Instantiates a new Create resources transform. * * @param fhirStore the fhir store - * @param type the type */ - CreateResources(ValueProvider fhirStore, String type) { + CreateResources(ValueProvider fhirStore) { this.fhirStore = fhirStore.get(); - this.type = type; } /** * Instantiates a new Create resources. * * @param fhirStore the fhir store - * @param type the type */ - CreateResources(String fhirStore, String type) { + CreateResources(String fhirStore) { this.fhirStore = fhirStore; - this.type = type; } /** @@ -1300,6 +1317,20 @@ public CreateResources withIfNotExistFunction( return this; } + /** + * This adds a {@link SerializableFunction} that reads an resource string and extracts an + * resource type. + * + *

    https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/create + * + * @param typeFunction for extracting type from a resource. + * @return the create resources + */ + public CreateResources withTypeFunction( + SerializableFunction typeFunction) { + this.typeFunction = typeFunction; + return this; + } /** * With format body function create resources. * @@ -1314,6 +1345,9 @@ public CreateResources withFormatBodyFunction( @Override public FhirIO.Write.Result expand(PCollection input) { + checkArgument( + typeFunction != null, + "FhirIO.CreateResources should always be called with a " + "withTypeFunction"); checkArgument( formatBodyFunction != null, "FhirIO.CreateResources should always be called with a " + "withFromatBodyFunction"); @@ -1327,7 +1361,7 @@ public FhirIO.Write.Result expand(PCollection input) { input .apply( ParDo.of( - new CreateFn(fhirStore, type, formatBodyFunction, ifNoneExistFunction))) + new CreateFn(fhirStore, typeFunction, formatBodyFunction, ifNoneExistFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1337,17 +1371,17 @@ static class CreateFn extends DoFn> { private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); private final String fhirStore; - private final String type; private SerializableFunction ifNoneExistFunction; private SerializableFunction formatBodyFunction; + private SerializableFunction typeFunction; CreateFn( String fhirStore, - String type, + SerializableFunction typeFunction, SerializableFunction formatBodyFunction, @Nullable SerializableFunction ifNoneExistFunction) { this.fhirStore = fhirStore; - this.type = type; + this.typeFunction = typeFunction; this.formatBodyFunction = formatBodyFunction; this.ifNoneExistFunction = ifNoneExistFunction; } @@ -1361,6 +1395,7 @@ public void initClient() throws IOException { public void create(ProcessContext context) { T input = context.element(); String body = formatBodyFunction.apply(input); + String type = typeFunction.apply(input); try { // Validate that data was set to valid JSON. mapper.readTree(input.toString()); @@ -1378,6 +1413,28 @@ public void create(ProcessContext context) { } } + /** + * Update resources update resources. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the update resources + */ + public static UpdateResources update(ValueProvider fhirStore){ + return new UpdateResources(fhirStore); + } + + /** + * Update resources update resources. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the update resources + */ + public static UpdateResources update(String fhirStore){ + return new UpdateResources(fhirStore); + } + /** * {@link PTransform} for Updating FHIR resources resources. * @@ -1488,6 +1545,28 @@ public void update(ProcessContext context) { } } + /** + * Conditional update conditional update. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the conditional update + */ + public static ConditionalUpdate conditionalUpdate(ValueProvider fhirStore){ + return new ConditionalUpdate(fhirStore); + } + + /** + * Conditional update conditional update. + * + * @param the type parameter + * @param fhirStore the fhir store + * @return the conditional update + */ + public static ConditionalUpdate conditionalUpdate(String fhirStore){ + return new ConditionalUpdate(fhirStore); + } + /** * {@link PTransform} to perform Conditional updates on the FHIR store. * diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 4bbea81cb87e..1c3a795fc2ec 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -44,8 +44,11 @@ class FhirIOTestUtil { public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests"; + // TODO read initial resources function. + // TODO read update resources function. + // TODO spot check resource update utility. private static Stream readPrettyBundles(String version) { - Path resourceDir = Paths.get("src", "test", "resources", version); + Path resourceDir = Paths.get("src", "test", "resources", "transactional_bundles",version); String absolutePath = resourceDir.toFile().getAbsolutePath(); File dir = new File(absolutePath); File[] fhirJsons = dir.listFiles(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java index e8e06c60493c..8d46186c472b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java @@ -25,12 +25,17 @@ import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.junit.After; import org.junit.AfterClass; @@ -128,4 +133,64 @@ public void testFhirIO_Import() { } // TODO(jaketf) add IT for conditional create, update, conditional update transforms. + @Test + public void testFhirIO_CreateResources() { + FhirIO.Write.Result writeResult = + (Result) pipeline + .apply(Create.of(BUNDLES.get(version))) + .apply(FhirIO.createResources(options.getFhirStore()) + .withTypeFunction((String resource) -> { + ObjectMapper mapper = new ObjectMapper(); + Map map = mapper.readValue(resource, Map.class); + return map.get("resourceType"); + + }) + .withIfNotExistFunction((String resource) -> { + ObjectMapper mapper = new ObjectMapper(); + Map map = mapper.readValue(resource, Map.class); + String id = map.get("id"); + return String.format("_id=%s", id); + }) + .withFormatBodyFunction((String x) -> x) + ); + + + // TODO have a known number of pre-condition failed case and assert it appears in error queue + PAssert.thatSingleton(writeResult.getFailedBodies().apply(Count.globally())).isEqualTo(1L); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void testFhirIO_Update() { + // TODO write initial resources to FHIR + // use beam to perform updates + FhirIO.Write.Result writeResult = + (Result) pipeline + .apply(Create.of(BUNDLES.get(version))) + .apply(FhirIO.update(options.getFhirStore()) + .withFormatBodyFunction(x -> "{}") + .withResourceNameFunction(x -> "name") + ); + + PAssert.that(writeResult.getFailedBodies()).empty(); + + pipeline.run().waitUntilFinish(); + // TODO spot check update results + } + + @Test + public void testFhirIO_ConditionalUpdate() { + // TODO write initial resources to FHIR + FhirIO.Write.Result writeResult = + (Result) pipeline + .apply(Create.of(BUNDLES.get(version))) + .apply(FhirIO.conditionalUpdate(options.getFhirStore()) + .withTypeFunction(x -> "patient") + .withFormatBodyFunction(x -> "{}") + .withSearchParametersFunction(x-> new HashMap()) + ); + // TODO spot check update results + // TODO have a known number of pre-condition failed case and assert it appears in error queue + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/CarePlan.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/CarePlan.ndjson new file mode 100644 index 000000000000..a4034d143b89 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/CarePlan.ndjson @@ -0,0 +1,3 @@ +{"resourceType":"CarePlan","id":"ba15e4c6-414b-4385-8abf-7be2eae77983","text":{"status":"generated","div":"

    Respiratory therapy
    "},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","context":{"reference":"Encounter/429ca9b0-f562-4845-aff6-0b6fc3aa1085"},"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-19T14:15:05-07:00"},"category":[{"coding":[{"system":"http://snomed.info/sct","code":"53950000","display":"Respiratory therapy"}],"text":"Respiratory therapy"}],"addresses":[{"reference":"Condition/46d525ba-6571-44de-a6b0-e43a3810bc8e"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"304510005","display":"Recommendation to avoid exercise"}],"text":"Recommendation to avoid exercise"},"status":"completed","prohibited":false}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"371605008","display":"Deep breathing and coughing exercises"}],"text":"Deep breathing and coughing exercises"},"status":"completed","prohibited":false}}]} +{"resourceType":"CarePlan","id":"c9f29319-842a-4528-8866-b915f558d66b","text":{"status":"generated","div":"
    Fracture care
    "},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","context":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"},"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-12-29T13:15:05-08:00"},"category":[{"coding":[{"system":"http://snomed.info/sct","code":"385691007","display":"Fracture care"}],"text":"Fracture care"}],"addresses":[{"reference":"Condition/b1cecfd7-b8b9-4159-98c8-95e7ad63d74a"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed","prohibited":false}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"408580007","display":"Physical activity target light exercise"}],"text":"Physical activity target light exercise"},"status":"completed","prohibited":false}}]} +{"resourceType":"CarePlan","id":"0b324605-d5b0-4356-a80b-59148979151a","text":{"status":"generated","div":"
    Minor surgery care management (procedure)
    "},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","context":{"reference":"Encounter/dd792a18-dcd6-4d70-ad52-78a04ab63018"},"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-23T13:15:05-08:00"},"category":[{"coding":[{"system":"http://snomed.info/sct","code":"737471002","display":"Minor surgery care management (procedure)"}],"text":"Minor surgery care management (procedure)"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed","prohibited":false}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"243077000","display":"Recommendation to limit sexual activity"}],"text":"Recommendation to limit sexual activity"},"status":"completed","prohibited":false}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Claim.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Claim.ndjson new file mode 100644 index 000000000000..49cc9d98c2b9 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Claim.ndjson @@ -0,0 +1,25 @@ +{"resourceType":"Claim","id":"fd5133df-a283-4ed2-a0a1-600c5f781504","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/759887f5-cdb3-42aa-ba72-403b3c1017ec"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"40f42d0e-2644-4b93-b81e-aa8333f5d107","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","diagnosis":[{"sequence":1,"diagnosis":{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}}]} +{"resourceType":"Claim","id":"c3d7130b-eab8-472f-86d6-21dd373ad796","type":"institutional","organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"},"net":{"value":504.83,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"f0059888-64fd-4177-b90f-dbb450d95914","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/e1582c1a-e002-40be-8e5e-041d71b32945"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"aea74302-dc46-4032-80b9-dfc41748edf3","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}}]} +{"resourceType":"Claim","id":"8e1db8cc-ce4c-426c-8421-a9d518a30a73","type":"institutional","organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"a53e8d15-f6ab-4c5a-b984-c60796304d9c","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/442bf25b-fd87-4bec-9c71-7fff3389f54d"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"7fab04c8-ea33-4cc3-a050-0b8008a50409","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}}]} +{"resourceType":"Claim","id":"98f46e66-f451-47db-81cc-f77bed1e183c","type":"institutional","organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":4,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":5,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"75510ac1-f51c-4fad-a255-711d8d1acfbe","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/2339a360-d66a-4518-848b-f2ebdf0db27e"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"89f97d6a-9511-4cf8-84fe-9c8369af8188","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}}]} +{"resourceType":"Claim","id":"ea6a35dc-0be4-4be9-90ec-88f58800b35b","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/417abd1f-43e3-446f-8b4e-3b237bc6c1ba"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"966e64bb-385a-4ac0-b97e-4cfa225ecea5","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","diagnosis":[{"sequence":1,"diagnosis":{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"},"net":{"value":6535.58,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"f731e722-2f3b-4c90-8488-2595ae8374b6","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"0b196692-83fa-4edd-beab-fc1a4e3a31b3","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/fe1a0af0-3bb4-4864-a7f6-8bc91a6f29e6"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"e1b59b3b-65ee-45f0-ae4d-5c69fadd3e94","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}}]} +{"resourceType":"Claim","id":"a975e6b7-b0ee-4df0-b1f3-406da196f899","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/8fcc3aa1-2288-4c60-9670-8c756295d7e1"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"37aa3b3c-0bd6-4756-bdba-a19c17b6f3ff","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/7dc37b2f-c065-48fc-a5e4-c0973fe705b7"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"2cd90bd3-cfd7-4ddc-a7dc-e5df0443f070","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","diagnosis":[{"sequence":1,"diagnosis":{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":4,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"02e90c71-8d21-4e91-8e0a-d948446ebddc","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}}]} +{"resourceType":"Claim","id":"24df9742-07df-44f8-a09d-823fee928063","type":"institutional","organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"},"net":{"value":482.02,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"617f658b-6244-4e9b-b53e-3edaa636b540","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}}]} +{"resourceType":"Claim","id":"8371259c-ea6c-458f-b7d1-4e18c36e976d","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}},{"sequence":2,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"},"net":{"value":9364.49,"system":"urn:iso:std:iso:4217","code":"USD"}}]} +{"resourceType":"Claim","id":"c135ee04-a1be-4b2e-9b47-1b0cb7939d6a","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","prescription":{"reference":"MedicationOrder/19a955e5-961c-4042-95fa-3d3ddefdd75d"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"}} +{"resourceType":"Claim","id":"4490461e-44df-42c1-ab4d-83bb8e081632","type":"institutional","organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"use":"complete","diagnosis":[{"sequence":1,"diagnosis":{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"item":[{"sequence":1,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}},{"sequence":3,"type":{"system":"http://hl7.org/fhir/v3/ActCode","code":"CSINV","display":"clinical service invoice"},"service":{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"},"net":{"value":1958.61,"system":"urn:iso:std:iso:4217","code":"USD"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Condition.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Condition.ndjson new file mode 100644 index 000000000000..03293ca003c5 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Condition.ndjson @@ -0,0 +1,4 @@ +{"resourceType":"Condition","id":"6468ef65-32eb-43c0-a292-498217bfe77d","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c596d869-aee8-47be-be1f-9044a0b03056"},"dateRecorded":"2010-05-19","code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"category":{"coding":[{"system":"http://hl7.org/fhir/condition-category","code":"diagnosis"}]},"clinicalStatus":"resolved","verificationStatus":"confirmed","onsetDateTime":"2010-05-19T14:15:05-07:00","abatementDateTime":"2010-05-26T14:15:05-07:00"} +{"resourceType":"Condition","id":"46d525ba-6571-44de-a6b0-e43a3810bc8e","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/429ca9b0-f562-4845-aff6-0b6fc3aa1085"},"dateRecorded":"2015-04-12","code":{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"},"category":{"coding":[{"system":"http://hl7.org/fhir/condition-category","code":"diagnosis"}]},"clinicalStatus":"resolved","verificationStatus":"confirmed","onsetDateTime":"2015-04-12T14:15:05-07:00","abatementDateTime":"2015-04-19T14:15:05-07:00"} +{"resourceType":"Condition","id":"b1cecfd7-b8b9-4159-98c8-95e7ad63d74a","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"},"dateRecorded":"2016-10-30","code":{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"},"category":{"coding":[{"system":"http://hl7.org/fhir/condition-category","code":"diagnosis"}]},"clinicalStatus":"resolved","verificationStatus":"confirmed","onsetDateTime":"2016-10-30T14:15:05-07:00","abatementDateTime":"2016-12-29T13:15:05-08:00"} +{"resourceType":"Condition","id":"83443d35-4ad0-43f9-9d04-a38fd1a40546","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/bfb92e78-2a6d-4a28-8cac-9511f1ec98ce"},"dateRecorded":"2018-12-11","code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"category":{"coding":[{"system":"http://hl7.org/fhir/condition-category","code":"diagnosis"}]},"clinicalStatus":"resolved","verificationStatus":"confirmed","onsetDateTime":"2018-12-11T13:15:05-08:00","abatementDateTime":"2018-12-19T13:15:05-08:00"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/DiagnosticReport.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/DiagnosticReport.ndjson new file mode 100644 index 000000000000..446b6c943cfe --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/DiagnosticReport.ndjson @@ -0,0 +1 @@ +{"resourceType":"DiagnosticReport","id":"202599a0-96e1-40d7-ad50-7759082e833b","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/ValueSet/diagnostic-service-sections","code":"LAB"}]},"code":{"coding":[{"system":"http://loinc.org","code":"58410-2","display":"Complete blood count (hemogram) panel - Blood by Automated count"}],"text":"Complete blood count (hemogram) panel - Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","performer":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"result":[{"reference":"Observation/af682f69-afe1-4d95-b2ff-701e99b48057","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Encounter.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Encounter.ndjson new file mode 100644 index 000000000000..53f484883471 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Encounter.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"Encounter","id":"c596d869-aee8-47be-be1f-9044a0b03056","status":"finished","class":"ambulatory","type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"2a0b33f5-4f48-468a-b814-16415492c421","status":"finished","type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"f1965c82-d450-4740-956c-1d5481c2bc3f","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"f85534f8-57c0-4c87-ba1a-9c51240b2749","status":"finished","type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"269a52f1-6e55-474b-b6f7-ad4c4acf4929","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"c416f8eb-c3b5-4c45-bfb8-4617e9050442","status":"finished","type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"ed094139-44fd-483c-bf6f-38fdbb1e2925","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"429ca9b0-f562-4845-aff6-0b6fc3aa1085","status":"finished","class":"ambulatory","type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"76264224-5098-4cef-8731-f96423f99704","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}],"text":"Encounter for check up (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"3e7926ca-2d44-4f15-a7a7-2879e7103e7a","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"0059d564-708d-467b-b8d8-449492ad93c7","status":"finished","class":"emergency","type":[{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"402eba83-dc9f-4be5-bf3f-a4c074d730e2","status":"finished","class":"ambulatory","type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}],"text":"Encounter for 'check-up'"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c","status":"finished","type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"3c725b99-002f-468c-accc-5037dc124942","status":"finished","class":"outpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"dd792a18-dcd6-4d70-ad52-78a04ab63018","status":"finished","class":"inpatient","type":[{"coding":[{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}],"text":"Admission to surgical department"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"bfb92e78-2a6d-4a28-8cac-9511f1ec98ce","status":"finished","class":"ambulatory","type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/ImagingStudy.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/ImagingStudy.ndjson new file mode 100644 index 000000000000..a59b691dd652 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/ImagingStudy.ndjson @@ -0,0 +1 @@ +{"resourceType":"ImagingStudy","id":"b981d431-fd72-4325-878f-1bcf0e3bc321","started":"2016-10-30T14:15:05-07:00","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"uid":"urn:oid:1.2.840.99999999.22502730.1589831196459","numberOfSeries":1,"numberOfInstances":1,"series":[{"number":1,"modality":{"system":"http://dicom.nema.org/resources/ontology/DCM","code":"DX","display":"Digital Radiography"},"uid":"urn:oid:1.2.840.99999999.1.65971777.1589831196459","numberOfInstances":1,"availability":"UNAVAILABLE","bodySite":{"system":"http://snomed.info/sct","code":"51299004","display":"Clavicle"},"started":"2016-10-30T14:15:05-07:00","instance":[{"number":1,"uid":"urn:oid:1.2.840.99999999.1.1.48010377.1589831196459","sopClass":"urn:oid:1.2.840.10008.5.1.4.1.1.1.1","title":"Image of clavicle"}]}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Immunization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Immunization.ndjson new file mode 100644 index 000000000000..fa03bd45f190 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Immunization.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"Immunization","id":"217e6566-fef3-4cfc-9bc1-b4dda86f4a7d","status":"completed","date":"2010-06-03T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"}} +{"resourceType":"Immunization","id":"0a185e37-eed6-4196-baeb-edb7d2334923","status":"completed","date":"2011-06-09T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"}} +{"resourceType":"Immunization","id":"388eac61-1d07-4787-bdac-8cdf0a7426a9","status":"completed","date":"2014-06-12T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"}} +{"resourceType":"Immunization","id":"92394c6c-5eb5-4169-ac5f-78dc240a0347","status":"completed","date":"2014-06-12T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"}],"text":"Td (adult) preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"}} +{"resourceType":"Immunization","id":"d9d6410c-a65e-4b6c-a2f2-1d55ac41a1b5","status":"completed","date":"2014-06-12T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"}} +{"resourceType":"Immunization","id":"37c95cd2-6fb8-4aad-9303-716f6cee7a60","status":"completed","date":"2014-06-12T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"}],"text":"meningococcal MCV4P"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"}} +{"resourceType":"Immunization","id":"b8be2715-bd14-4627-adbf-70d0b27a2c9e","status":"completed","date":"2015-04-23T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"}} +{"resourceType":"Immunization","id":"f7a65385-4028-470a-a365-ee9836782642","status":"completed","date":"2015-04-23T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"}} +{"resourceType":"Immunization","id":"e93c7361-bfd5-459b-ad13-de4b0bbfaa45","status":"completed","date":"2017-06-15T14:15:05-07:00","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"wasNotGiven":false,"reported":false,"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/MedicationOrder.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/MedicationOrder.ndjson new file mode 100644 index 000000000000..a29d53277cff --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/MedicationOrder.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"MedicationOrder","id":"759887f5-cdb3-42aa-ba72-403b3c1017ec","dateWritten":"2010-05-19T14:15:05-07:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/c596d869-aee8-47be-be1f-9044a0b03056"},"reasonReference":{"reference":"Condition/6468ef65-32eb-43c0-a292-498217bfe77d"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"}} +{"resourceType":"MedicationOrder","id":"e1582c1a-e002-40be-8e5e-041d71b32945","dateWritten":"2011-03-11T13:15:05-08:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/f1965c82-d450-4740-956c-1d5481c2bc3f"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"751905","display":"Trinessa 28 Day Pack"}],"text":"Trinessa 28 Day Pack"}} +{"resourceType":"MedicationOrder","id":"442bf25b-fd87-4bec-9c71-7fff3389f54d","dateWritten":"2013-02-28T13:15:05-08:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/269a52f1-6e55-474b-b6f7-ad4c4acf4929"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"748856","display":"Yaz 28 Day Pack"}],"text":"Yaz 28 Day Pack"}} +{"resourceType":"MedicationOrder","id":"2339a360-d66a-4518-848b-f2ebdf0db27e","dateWritten":"2015-02-18T13:15:05-08:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/ed094139-44fd-483c-bf6f-38fdbb1e2925"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"1534809","display":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"}],"text":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"}} +{"resourceType":"MedicationOrder","id":"417abd1f-43e3-446f-8b4e-3b237bc6c1ba","dateWritten":"2015-04-12T14:15:05-07:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/429ca9b0-f562-4845-aff6-0b6fc3aa1085"},"reasonReference":{"reference":"Condition/46d525ba-6571-44de-a6b0-e43a3810bc8e"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"313782","display":"Acetaminophen 325 MG Oral Tablet"}],"text":"Acetaminophen 325 MG Oral Tablet"}} +{"resourceType":"MedicationOrder","id":"fe1a0af0-3bb4-4864-a7f6-8bc91a6f29e6","dateWritten":"2016-02-13T13:15:05-08:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/3e7926ca-2d44-4f15-a7a7-2879e7103e7a"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"978950","display":"Natazia 28 Day Pack"}],"text":"Natazia 28 Day Pack"}} +{"resourceType":"MedicationOrder","id":"8fcc3aa1-2288-4c60-9670-8c756295d7e1","dateWritten":"2016-10-30T14:15:05-07:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"861467","display":"Meperidine Hydrochloride 50 MG Oral Tablet"}],"text":"Meperidine Hydrochloride 50 MG Oral Tablet"},"dosageInstruction":[{"timing":{"repeat":{"frequency":1,"period":4.0,"periodUnits":"h"}},"asNeededBoolean":false,"doseQuantity":{"value":1.0}}]} +{"resourceType":"MedicationOrder","id":"7dc37b2f-c065-48fc-a5e4-c0973fe705b7","dateWritten":"2016-10-30T14:15:05-07:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"310965","display":"Ibuprofen 200 MG Oral Tablet"}],"text":"Ibuprofen 200 MG Oral Tablet"},"dosageInstruction":[{"asNeededBoolean":true}]} +{"resourceType":"MedicationOrder","id":"19a955e5-961c-4042-95fa-3d3ddefdd75d","dateWritten":"2018-12-11T13:15:05-08:00","status":"stopped","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"prescriber":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"encounter":{"reference":"Encounter/bfb92e78-2a6d-4a28-8cac-9511f1ec98ce"},"reasonReference":{"reference":"Condition/83443d35-4ad0-43f9-9d04-a38fd1a40546"},"medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Observation.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Observation.ndjson new file mode 100644 index 000000000000..a139a7c21cc7 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Observation.ndjson @@ -0,0 +1,54 @@ +{"resourceType":"Observation","id":"ace52964-c945-4434-8bce-7dc8334ebb1f","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":153.6,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"280ac889-62a2-4f40-b1df-3ee1bf55c44a","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":1,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"5c69fc59-93ae-4569-881a-90c95e713783","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":55.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"da3b05a9-7c2e-47bb-99a6-c292e7e3410a","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":23.41,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"b3ff3362-0faa-4d82-adbb-c2e9abf66b26","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":71.788,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"fa401b31-c5c0-4773-9f2f-9e2ac2faf530","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":72,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":120,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"5dd09901-1878-4c33-8bea-901867ab8ef6","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"14727409-d62b-448d-8739-f7a34fbda73d","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"1049a0e2-da6e-4972-a1bb-59255b4f0915","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}],"text":"survey"},"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"6c52e341-718e-48a6-9619-91ec7fe15a71","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":153.7,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"e3e7db52-6dfe-4f72-9970-387f6ddd1103","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"f315729d-f344-4fb0-a67c-e2acdb128bb3","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":62.1,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"4d63298f-9a97-44e1-814c-5d82dbb662d5","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":26.3,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"f4823f99-c40c-4a5b-84da-0c581f6b0510","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":85.491,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"0a550125-9dae-4acb-a4c6-5464fe580dbe","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":88,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":121,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"19aced8c-1ed9-4dbe-b829-9499fe1e8324","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":93,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"65b982c9-1a37-40ba-a459-06f57d0e8272","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"b4e25ecd-57bc-4f7a-b73c-038292bbc60a","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}],"text":"survey"},"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f85534f8-57c0-4c87-ba1a-9c51240b2749"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"7bd7f85e-ddca-4d45-b813-7d92169da64e","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"d93dc239-a2bf-4a59-8379-26ccb39cfe67","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"4402d329-81a3-41ae-9179-3b23a4896f26","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":69.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"fefb4870-44be-4f80-8f1e-c1ef963e22d1","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":29.25,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"01a3e5b1-f2cf-4aa7-85d0-9c99610e9e7f","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":87,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":122,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"0a1c6fb9-1614-4e7e-9c8c-5aaf3da2897f","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":90,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"8a7a8549-1260-4e5f-bfaa-ba27f27b44ca","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":16,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"6a7cd9eb-829d-4771-8109-b2234a7b0c49","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"6690-2","display":"Leukocytes [#/volume] in Blood by Automated count"}],"text":"Leukocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":7.2857,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"097a47bf-7ae9-471f-a081-87b165c81f63","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"789-8","display":"Erythrocytes [#/volume] in Blood by Automated count"}],"text":"Erythrocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":4.7109,"unit":"10*6/uL","system":"http://unitsofmeasure.org","code":"10*6/uL"}} +{"resourceType":"Observation","id":"6c6590f8-3fbc-4cdd-93fd-d40ab785abbf","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"718-7","display":"Hemoglobin [Mass/volume] in Blood"}],"text":"Hemoglobin [Mass/volume] in Blood"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":15.964,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"4c1c503e-70b7-4d4e-8751-762c16a202a2","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"4544-3","display":"Hematocrit [Volume Fraction] of Blood by Automated count"}],"text":"Hematocrit [Volume Fraction] of Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":45.934,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"1b89993a-e7ae-4620-a10c-7aeb598a75a9","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"787-2","display":"MCV [Entitic volume] by Automated count"}],"text":"MCV [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":94.793,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"78a08462-e203-4ec1-a9ce-51b29262080b","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"785-6","display":"MCH [Entitic mass] by Automated count"}],"text":"MCH [Entitic mass] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":28.937,"unit":"pg","system":"http://unitsofmeasure.org","code":"pg"}} +{"resourceType":"Observation","id":"ba5f4798-0ea9-451e-a943-99d048d05c0b","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"786-4","display":"MCHC [Mass/volume] by Automated count"}],"text":"MCHC [Mass/volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":33.479,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"621b2d18-f34d-446d-b35e-dfefc0acb826","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"21000-5","display":"Erythrocyte distribution width [Entitic volume] by Automated count"}],"text":"Erythrocyte distribution width [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":43.303,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"512bdee3-dc80-41dc-8d15-c220a7504ebf","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"777-3","display":"Platelets [#/volume] in Blood by Automated count"}],"text":"Platelets [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":173.89,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"1ab93bb4-1867-432f-b5d7-002a43b314a1","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"32207-3","display":"Platelet distribution width [Entitic volume] in Blood by Automated count"}],"text":"Platelet distribution width [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":413.95,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"af682f69-afe1-4d95-b2ff-701e99b48057","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}],"text":"laboratory"},"code":{"coding":[{"system":"http://loinc.org","code":"32623-1","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}],"text":"Platelet mean volume [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":10.07,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"1c2bfa7f-80e8-430e-922b-6d0d51f409d6","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}],"text":"survey"},"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/c416f8eb-c3b5-4c45-bfb8-4617e9050442"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"3fd81e92-32c5-4516-b66b-2bb3abe1ae6c","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"ce4b4ec2-8b75-4270-8822-52e801961e88","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"d47d450a-a881-4aa7-8281-37059113722f","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":70.3,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"c78bef2c-6c68-4039-b1d9-1dbdd18b648b","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":29.73,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"4c6b48d2-8b38-47e6-868f-a7e38888a471","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":79,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":129,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"2737d9c5-60b3-471a-a5d0-f5e2b5763485","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":90,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"23762ec7-dba6-4114-96d1-0b8f91d28194","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":15,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"fb274daa-b9a8-423f-b10a-9f74ec1bf383","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}],"text":"survey"},"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/76264224-5098-4cef-8731-f96423f99704"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"2b7d5771-4912-4901-ba2d-ed4956f549cc","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"74f833a2-5b10-4482-a8f7-7b672b54fdfd","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":0,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"2958f9d7-227b-476b-8d6f-0d96dea1b2fd","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66.8,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"bb5ffa55-1a1d-4b8b-8ccc-39bcbc38af58","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":28.22,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"fda5887c-49ba-4494-b263-bd6c5124f3b8","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":69,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":111,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"0853005c-9c07-4da4-b91a-22e4fb4145b7","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"d960be5d-3b83-4a5a-8f8b-653c9fdd79b7","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":12,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"131cd0ab-612d-4b9e-97b8-db72573d684d","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}],"text":"survey"},"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"fe4d9e94-f3ff-48b0-a731-4f7fd1a68ad3","status":"final","category":{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}],"text":"vital-signs"},"code":{"coding":[{"system":"http://loinc.org","code":"8310-5","display":"Body temperature"}],"text":"Body temperature"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/bfb92e78-2a6d-4a28-8cac-9511f1ec98ce"},"effectiveDateTime":"2018-12-11T13:15:05-08:00","issued":"2018-12-11T13:15:05.225-08:00","valueQuantity":{"value":39.024,"unit":"Cel","system":"http://unitsofmeasure.org","code":"Cel"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Organization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Organization.ndjson new file mode 100644 index 000000000000..6e8753228fa1 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Organization.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Organization","id":"465de31f-3098-365c-af70-48a071e1f5aa","identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"465de31f-3098-365c-af70-48a071e1f5aa"}],"type":{"coding":[{"system":"Healthcare Provider","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"},"name":"METROWEST MEDICAL CENTER","telecom":[{"system":"phone","value":"5083831000"}],"address":[{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}]} +{"resourceType":"Organization","id":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}],"type":{"coding":[{"system":"Healthcare Provider","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"},"name":"PCP68975","telecom":[{"system":"phone","value":"508-881-4368"}],"address":[{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Patient.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Patient.ndjson new file mode 100644 index 000000000000..d8d4f4a9c389 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Patient.ndjson @@ -0,0 +1 @@ +{"resourceType":"Patient","id":"1416dec1-f4b1-4b48-b7f4-650e8f67499c","text":{"status":"generated","div":"
    Generated by Synthea.Version identifier: v2.5.0-265-gbd5a00e8\n . Person seed: 6732543839779682504 Population seed: 1589831189867
    "},"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/us-core-race","valueCodeableConcept":{"coding":[{"system":"http://hl7.org/fhir/v3/Race","code":"2106-3","display":"White"}],"text":"White"}},{"url":"http://hl7.org/fhir/StructureDefinition/us-core-ethnicity","valueCodeableConcept":{"coding":[{"system":"http://hl7.org/fhir/v3/Ethnicity","code":"2186-5","display":"Not Hispanic or Latino"}],"text":"Not Hispanic or Latino"}},{"url":"http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName","valueString":"Leanna255 Predovic534"},{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex","valueCode":"F"},{"url":"http://hl7.org/fhir/StructureDefinition/birthPlace","valueAddress":{"city":"Southbridge","state":"Massachusetts","country":"US"}},{"url":"http://synthetichealth.github.io/synthea/disability-adjusted-life-years","valueDecimal":0.0},{"url":"http://synthetichealth.github.io/synthea/quality-adjusted-life-years","valueDecimal":27.0}],"identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://hl7.org/fhir/v2/0203","code":"MR"}]},"system":"http://hospital.smarthealthit.org","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://hl7.org/fhir/identifier-type","code":"SB"}]},"system":"http://hl7.org/fhir/sid/us-ssn","value":"999-54-3579"},{"type":{"coding":[{"system":"http://hl7.org/fhir/v2/0203","code":"DL"}]},"system":"urn:oid:2.16.840.1.113883.4.3.25","value":"S99972984"}],"name":[{"use":"official","family":["Nolan344"],"given":["Lorita217"],"prefix":["Ms."]}],"telecom":[{"system":"phone","value":"555-817-6998","use":"home"}],"gender":"female","birthDate":"1992-04-09","address":[{"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/geolocation","extension":[{"url":"latitude","valueDecimal":42.27693107900605},{"url":"longitude","valueDecimal":-71.45741653702677}]}],"line":["330 Sawayn Parade"],"city":"Framingham","state":"Massachusetts","country":"US"}],"maritalStatus":{"coding":[{"system":"http://hl7.org/fhir/v3/MaritalStatus","code":"S"}]},"multipleBirthBoolean":false,"communication":[{"language":{"coding":[{"system":"urn:ietf:bcp:47","code":"en-US","display":"English"}],"text":"English"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Practitioner.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Practitioner.ndjson new file mode 100644 index 000000000000..e40c5c9d1f46 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Practitioner.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Practitioner","id":"c16820ae-2954-32d4-863c-e9ceb741154c","identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"530"}],"active":true,"name":{"family":["Murphy561"],"given":["Mari763"],"prefix":["Dr."]},"address":[{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}],"gender":"female"} +{"resourceType":"Practitioner","id":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"35750"}],"active":true,"name":{"family":["Hilpert278"],"given":["Cathryn51"],"prefix":["Dr."]},"address":[{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}],"gender":"female"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Procedure.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Procedure.ndjson new file mode 100644 index 000000000000..c86efacdeb1b --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/Procedure.ndjson @@ -0,0 +1,7 @@ +{"resourceType":"Procedure","id":"c5141e34-d584-4900-b59c-ef099efe22dc","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"performedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:30:05-07:00"},"encounter":{"reference":"Encounter/2a0b33f5-4f48-468a-b814-16415492c421"}} +{"resourceType":"Procedure","id":"2349f028-54e4-43fd-8060-2ce6f810924b","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"}],"text":"Plain chest X-ray (procedure)"},"reasonReference":{"reference":"Condition/46d525ba-6571-44de-a6b0-e43a3810bc8e"},"performedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:26:05-07:00"},"encounter":{"reference":"Encounter/429ca9b0-f562-4845-aff6-0b6fc3aa1085"}} +{"resourceType":"Procedure","id":"ef872ffd-e080-4432-bd24-1d41aef41ed7","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"}],"text":"Clavicle X-ray"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T14:45:05-07:00"},"encounter":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"}} +{"resourceType":"Procedure","id":"61a3a3f6-bcad-49ef-ab79-ea87fab516cc","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"}],"text":"Admission to orthopedic department"},"reasonReference":{"reference":"Condition/b1cecfd7-b8b9-4159-98c8-95e7ad63d74a"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T15:15:05-07:00"},"encounter":{"reference":"Encounter/0059d564-708d-467b-b8d8-449492ad93c7"}} +{"resourceType":"Procedure","id":"cdfebd64-8e72-4aec-bd6a-2be27eb41b82","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"performedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:30:05-07:00"},"encounter":{"reference":"Encounter/6ca65d01-35b1-4b58-8ee7-d4a6cedd2f5c"}} +{"resourceType":"Procedure","id":"b34c7fd4-6bb2-4c62-9d3c-98aa4e125251","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"}],"text":"Bilateral tubal ligation"},"performedPeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-09T15:15:05-08:00"},"encounter":{"reference":"Encounter/dd792a18-dcd6-4d70-ad52-78a04ab63018"}} +{"resourceType":"Procedure","id":"edf461c8-c51f-4161-bc66-7b61e12627af","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"}],"text":"Throat culture (procedure)"},"reasonReference":{"reference":"Condition/83443d35-4ad0-43f9-9d04-a38fd1a40546"},"performedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:30:05-08:00"},"encounter":{"reference":"Encounter/bfb92e78-2a6d-4a28-8cac-9511f1ec98ce"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json new file mode 100644 index 000000000000..9cf9888c287a --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json @@ -0,0 +1,183 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", + "resource": { + "resourceType": "Organization", + "id": "465de31f-3098-365c-af70-48a071e1f5aa", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "465de31f-3098-365c-af70-48a071e1f5aa" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "METROWEST MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5083831000" + } + ], + "address": [ + { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "resource": { + "resourceType": "Organization", + "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 23 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 5 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 17 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "PCP68975", + "telecom": [ + { + "system": "phone", + "value": "508-881-4368" + } + ], + "address": [ + { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "resource": { + "resourceType": "Organization", + "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "WALTHAM URGENT CARE", + "telecom": [ + { + "system": "phone", + "value": "617-243-5591" + } + ], + "address": [ + { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ] + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json new file mode 100644 index 000000000000..76f865f73c7c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json @@ -0,0 +1,135 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", + "resource": { + "resourceType": "Practitioner", + "id": "c16820ae-2954-32d4-863c-e9ceb741154c", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "530" + } + ], + "active": true, + "name": { + "family": [ + "Murphy561" + ], + "given": [ + "Mari763" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "resource": { + "resourceType": "Practitioner", + "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "35750" + } + ], + "active": true, + "name": { + "family": [ + "Hilpert278" + ], + "given": [ + "Cathryn51" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", + "resource": { + "resourceType": "Practitioner", + "id": "861aa935-d89f-3b41-861b-b791fa112253", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93440" + } + ], + "active": true, + "name": { + "family": [ + "Reynolds644" + ], + "given": [ + "Salena230" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ], + "gender": "female" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CarePlan.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CarePlan.ndjson new file mode 100644 index 000000000000..12ba93cc62a8 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CarePlan.ndjson @@ -0,0 +1,3 @@ +{"resourceType":"CarePlan","id":"32841402-d5d8-4615-af90-473bf6f417fa","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careplan"]},"text":{"status":"generated","div":"
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    "},"status":"completed","intent":"order","category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/careplan-category","code":"assess-plan"}]},{"coding":[{"system":"http://snomed.info/sct","code":"53950000","display":"Respiratory therapy"}],"text":"Respiratory therapy"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-19T14:15:05-07:00"},"careTeam":[{"reference":"CareTeam/111c8459-549e-4a2b-ab70-27f3b2408eac"}],"addresses":[{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"304510005","display":"Recommendation to avoid exercise"}],"text":"Recommendation to avoid exercise"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"371605008","display":"Deep breathing and coughing exercises"}],"text":"Deep breathing and coughing exercises"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}}]} +{"resourceType":"CarePlan","id":"ac2893f1-9c90-437d-852f-4cc1a0155d39","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careplan"]},"text":{"status":"generated","div":"
    Care Plan for Fracture care.
    Activities:
    • Fracture care
    • Fracture care

    Care plan is meant to treat Fracture of clavicle.
    "},"status":"completed","intent":"order","category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/careplan-category","code":"assess-plan"}]},{"coding":[{"system":"http://snomed.info/sct","code":"385691007","display":"Fracture care"}],"text":"Fracture care"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-12-29T13:15:05-08:00"},"careTeam":[{"reference":"CareTeam/31d5cf9a-c745-4087-8042-6a9b9babaef9"}],"addresses":[{"reference":"Condition/115098be-c26e-47d0-9cad-be9df75d7042"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"408580007","display":"Physical activity target light exercise"}],"text":"Physical activity target light exercise"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}}]} +{"resourceType":"CarePlan","id":"5a1fb9d1-310b-4fe3-b7c3-6370529400b8","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careplan"]},"text":{"status":"generated","div":"
    Care Plan for Minor surgery care management (procedure).
    Activities:
    • Minor surgery care management (procedure)
    • Minor surgery care management (procedure)
    "},"status":"completed","intent":"order","category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/careplan-category","code":"assess-plan"}]},{"coding":[{"system":"http://snomed.info/sct","code":"737471002","display":"Minor surgery care management (procedure)"}],"text":"Minor surgery care management (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"},"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-23T13:15:05-08:00"},"careTeam":[{"reference":"CareTeam/e1ab345a-81bd-461e-921d-9eae42c47f8a"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"243077000","display":"Recommendation to limit sexual activity"}],"text":"Recommendation to limit sexual activity"},"status":"completed","location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CareTeam.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CareTeam.ndjson new file mode 100644 index 000000000000..1bd6a0f44212 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/CareTeam.ndjson @@ -0,0 +1,3 @@ +{"resourceType":"CareTeam","id":"111c8459-549e-4a2b-ab70-27f3b2408eac","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careteam"]},"status":"inactive","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-19T14:15:05-07:00"},"participant":[{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"116153009","display":"Patient"}],"text":"Patient"}],"member":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Person in the healthcare environment (person)"}],"text":"Person in the healthcare environment (person)"}],"member":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Healthcare related organization (qualifier value)"}],"text":"Healthcare related organization (qualifier value)"}],"member":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}}],"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"}],"managingOrganization":[{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}]} +{"resourceType":"CareTeam","id":"31d5cf9a-c745-4087-8042-6a9b9babaef9","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careteam"]},"status":"inactive","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-12-29T13:15:05-08:00"},"participant":[{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"116153009","display":"Patient"}],"text":"Patient"}],"member":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Person in the healthcare environment (person)"}],"text":"Person in the healthcare environment (person)"}],"member":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Healthcare related organization (qualifier value)"}],"text":"Healthcare related organization (qualifier value)"}],"member":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}}],"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"}],"managingOrganization":[{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}]} +{"resourceType":"CareTeam","id":"e1ab345a-81bd-461e-921d-9eae42c47f8a","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-careteam"]},"status":"inactive","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"},"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-23T13:15:05-08:00"},"participant":[{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"116153009","display":"Patient"}],"text":"Patient"}],"member":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Person in the healthcare environment (person)"}],"text":"Person in the healthcare environment (person)"}],"member":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}},{"role":[{"coding":[{"system":"http://snomed.info/sct","code":"303118004","display":"Healthcare related organization (qualifier value)"}],"text":"Healthcare related organization (qualifier value)"}],"member":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}}],"managingOrganization":[{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Claim.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Claim.ndjson new file mode 100644 index 000000000000..05a7d0cf7222 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Claim.ndjson @@ -0,0 +1,25 @@ +{"resourceType":"Claim","id":"e24412c9-d798-4a45-8d29-98027d1eaa5d","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"created":"2010-05-19T14:45:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/8a21a17d-b0b3-48b2-8427-5eb9d1138628"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"}]}],"total":{"value":10.45,"currency":"USD"}} +{"resourceType":"Claim","id":"a7b49375-f326-462c-91a6-fb68c3d7c359","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"created":"2010-05-19T14:45:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"}]},{"sequence":2,"diagnosisSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"d4f07a36-012e-4170-8757-0c9bfbdfa575","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"created":"2010-06-03T14:45:05-07:00","provider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"supportingInfo":[{"sequence":1,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/f28a9afd-a143-45db-8aeb-082b0df22426"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/eba3e367-30ea-418d-ae97-e91506dea85f"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"encounter":[{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"}]},{"sequence":2,"informationSequence":[1],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"currency":"USD"}},{"sequence":3,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"net":{"value":504.83,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"e4ae0e04-c0b5-47c0-9f1c-1a4cefe48a97","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"created":"2011-03-11T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/c9954767-c8b7-45e0-8b07-099351dcd9a0"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"}]}],"total":{"value":31.0,"currency":"USD"}} +{"resourceType":"Claim","id":"6556aaa0-bf9f-40e7-bc54-1c3ceec565c7","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"created":"2011-03-11T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"d8f548d1-bf09-4fb3-a0d0-5454feee2396","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"created":"2011-06-09T14:30:05-07:00","provider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"supportingInfo":[{"sequence":1,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/e755857d-8ce3-42b6-81f0-d5b52524fae5"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"encounter":[{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"}]},{"sequence":2,"informationSequence":[1],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"2d2de9d8-9f05-4979-b19f-62d0733cbe9b","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"created":"2013-02-28T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/d10f9ae0-ffef-4a9b-8021-74a6cd6b1ad5"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"}]}],"total":{"value":26.21,"currency":"USD"}} +{"resourceType":"Claim","id":"171190a5-7d0d-45a4-99ac-0c519b6dd270","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"created":"2013-02-28T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"50387d60-9d96-475d-9719-7b9cb30071ca","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"created":"2014-06-12T14:30:05-07:00","provider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"supportingInfo":[{"sequence":1,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/490a18a8-30e4-4a65-9a8c-0993671cf186"}},{"sequence":2,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/e451e36e-dc92-4fd9-a924-1a83a04da939"}},{"sequence":3,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/673cd32a-bf63-4c7a-9e57-1b92d9fb15e9"}},{"sequence":4,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/41daff0e-fb2b-4216-8065-1481d7c962ed"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"encounter":[{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"}]},{"sequence":2,"informationSequence":[1],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"currency":"USD"}},{"sequence":3,"informationSequence":[2],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"}],"text":"Td (adult) preservative free"},"net":{"value":140.52,"currency":"USD"}},{"sequence":4,"informationSequence":[3],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"net":{"value":140.52,"currency":"USD"}},{"sequence":5,"informationSequence":[4],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"}],"text":"meningococcal MCV4P"},"net":{"value":140.52,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"39917e96-c35b-4c72-9abf-222dacb4d19e","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"created":"2015-02-18T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/c316061f-f9ad-4a30-80cf-c488b64b1112"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"}]}],"total":{"value":45.32,"currency":"USD"}} +{"resourceType":"Claim","id":"43604c79-af2f-4ea3-a871-a6efeb148686","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"created":"2015-02-18T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"9e1c8ca0-e622-46c2-ae92-5b32f9ebfff0","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"created":"2015-04-12T14:41:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/4b2bf1a7-6801-4ed0-85d2-336106458fdc"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"}]}],"total":{"value":5.32,"currency":"USD"}} +{"resourceType":"Claim","id":"5c4943b7-52fe-43cf-b800-200585136db8","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"created":"2015-04-12T14:41:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/78d69a34-6c80-4a6c-abbb-0accca0a17a5"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"}]},{"sequence":2,"diagnosisSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"}},{"sequence":3,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"}],"text":"Plain chest X-ray (procedure)"},"net":{"value":6535.58,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"99fc6fc4-98f4-415a-a094-f22df454163f","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"created":"2015-04-23T14:30:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"supportingInfo":[{"sequence":1,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/aebc3fce-9411-4246-87ee-17a65f3f5f9a"}},{"sequence":2,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/ab327672-36ba-4980-83a8-c4a1525de444"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}],"text":"Encounter for check up (procedure)"},"encounter":[{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"}]},{"sequence":2,"informationSequence":[1],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"currency":"USD"}},{"sequence":3,"informationSequence":[2],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"net":{"value":140.52,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"8c0e3992-24e7-4c44-9c50-3f0ea1431726","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"created":"2016-02-13T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/1df5e2a6-88ef-423c-ae94-5972e6ed7d36"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"}]}],"total":{"value":31.25,"currency":"USD"}} +{"resourceType":"Claim","id":"4a41154a-06aa-4bfc-94ce-b96f28f11b80","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"created":"2016-02-13T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"80484d29-eded-431d-bf9b-8a917089a77e","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"created":"2016-10-30T16:45:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/7202acd4-409c-4078-87bb-125f9ffe4bad"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"},"encounter":[{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"}]}],"total":{"value":113.81,"currency":"USD"}} +{"resourceType":"Claim","id":"41a00b32-22ff-4fd3-a388-296e4cc77dfe","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"created":"2016-10-30T16:45:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/edfc50e7-5843-4cbe-b4d2-b8a0c5b6bcfe"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"},"encounter":[{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"}]}],"total":{"value":17.35,"currency":"USD"}} +{"resourceType":"Claim","id":"07ae31e4-8480-4293-b506-5e9b32e7491b","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"created":"2016-10-30T16:45:05-07:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/115098be-c26e-47d0-9cad-be9df75d7042"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/e614b129-913f-42ac-bcc3-266db50af307"}},{"sequence":2,"procedureReference":{"reference":"Procedure/cf1cfcd7-ca27-4bf1-b8f0-14e31c69824e"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"},"encounter":[{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"}]},{"sequence":2,"diagnosisSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"}},{"sequence":3,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"}],"text":"Clavicle X-ray"},"net":{"value":516.65,"currency":"USD"}},{"sequence":4,"procedureSequence":[2],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"}],"text":"Admission to orthopedic department"},"net":{"value":516.65,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"b60311e1-94d9-473c-bf5f-ea06db1dad15","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"created":"2016-12-29T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}],"text":"Encounter for 'check-up'"},"encounter":[{"reference":"Encounter/a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"b52c9dfa-56bb-4523-9ace-38ad614ee107","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"created":"2017-06-15T14:45:05-07:00","provider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"supportingInfo":[{"sequence":1,"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/bc61721c-4053-4a81-b1e5-b64194e1efa1"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/ea19efb7-6346-45e9-aa8b-3195cc0b8aee"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"encounter":[{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"}]},{"sequence":2,"informationSequence":[1],"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"net":{"value":140.52,"currency":"USD"}},{"sequence":3,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"net":{"value":482.02,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"084d3b6a-e1d4-4e25-b90b-c970fbc20eb7","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"created":"2018-02-02T13:30:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"encounter":[{"reference":"Encounter/eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"}]}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"cf98de5a-9843-43c6-84c6-e6e703ffa457","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"created":"2018-02-10T15:15:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/91112614-fd2f-46cb-ad49-96111084ef1c"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}],"text":"Admission to surgical department"},"encounter":[{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"}]},{"sequence":2,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"}],"text":"Bilateral tubal ligation"},"net":{"value":9364.49,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} +{"resourceType":"Claim","id":"bfc5accc-20be-4556-b76a-e076fc4b2278","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"pharmacy"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"created":"2018-12-11T13:45:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"prescription":{"reference":"MedicationRequest/86aaf462-e2c8-4cdd-8db1-5edb78f15ea1"},"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"}]}],"total":{"value":16.98,"currency":"USD"}} +{"resourceType":"Claim","id":"15be99a3-ce23-43fd-a32a-fbda483a1224","status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Lorita217 Nolan344"},"billablePeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"created":"2018-12-11T13:45:05-08:00","provider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"priority":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/processpriority","code":"normal"}]},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/9c0af3aa-42c9-43a6-8677-ae5fa3f34922"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/1b7f052f-0bb7-422e-86c9-0c8f6af49fe2"}}],"insurance":[{"sequence":1,"focal":true,"coverage":{"display":"Medicaid"}}],"item":[{"sequence":1,"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"encounter":[{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"}]},{"sequence":2,"diagnosisSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"}},{"sequence":3,"procedureSequence":[1],"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"}],"text":"Throat culture (procedure)"},"net":{"value":1958.61,"currency":"USD"}}],"total":{"value":129.16,"currency":"USD"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Condition.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Condition.ndjson new file mode 100644 index 000000000000..138f07decf90 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Condition.ndjson @@ -0,0 +1,4 @@ +{"resourceType":"Condition","id":"4336ae40-60ee-4148-9271-a372cb9600a7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-condition"]},"clinicalStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-clinical","code":"resolved"}]},"verificationStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-ver-status","code":"confirmed"}]},"category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-category","code":"encounter-diagnosis","display":"Encounter Diagnosis"}]}],"code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"},"onsetDateTime":"2010-05-19T14:15:05-07:00","abatementDateTime":"2010-05-26T14:15:05-07:00","recordedDate":"2010-05-19T14:15:05-07:00"} +{"resourceType":"Condition","id":"c8d97153-17c7-4a2b-bafd-052ff8f30eb9","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-condition"]},"clinicalStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-clinical","code":"resolved"}]},"verificationStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-ver-status","code":"confirmed"}]},"category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-category","code":"encounter-diagnosis","display":"Encounter Diagnosis"}]}],"code":{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"onsetDateTime":"2015-04-12T14:15:05-07:00","abatementDateTime":"2015-04-19T14:15:05-07:00","recordedDate":"2015-04-12T14:15:05-07:00"} +{"resourceType":"Condition","id":"115098be-c26e-47d0-9cad-be9df75d7042","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-condition"]},"clinicalStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-clinical","code":"resolved"}]},"verificationStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-ver-status","code":"confirmed"}]},"category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-category","code":"encounter-diagnosis","display":"Encounter Diagnosis"}]}],"code":{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"onsetDateTime":"2016-10-30T14:15:05-07:00","abatementDateTime":"2016-12-29T13:15:05-08:00","recordedDate":"2016-10-30T14:15:05-07:00"} +{"resourceType":"Condition","id":"9c0af3aa-42c9-43a6-8677-ae5fa3f34922","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-condition"]},"clinicalStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-clinical","code":"resolved"}]},"verificationStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-ver-status","code":"confirmed"}]},"category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/condition-category","code":"encounter-diagnosis","display":"Encounter Diagnosis"}]}],"code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},"onsetDateTime":"2018-12-11T13:15:05-08:00","abatementDateTime":"2018-12-19T13:15:05-08:00","recordedDate":"2018-12-11T13:15:05-08:00"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DiagnosticReport.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DiagnosticReport.ndjson new file mode 100644 index 000000000000..7d7861955132 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DiagnosticReport.ndjson @@ -0,0 +1,17 @@ +{"resourceType":"DiagnosticReport","id":"c31327f8-f2ec-4b26-a8bd-40aff65cc085","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"},"effectiveDateTime":"2010-05-19T14:15:05-07:00","issued":"2010-05-19T14:15:05.225-07:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTAtMDUtMTkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCk5vIEFjdGl2ZSBNZWRpY2F0aW9ucy4KCiMgQXNzZXNzbWVudCBhbmQgUGxhbgpQYXRpZW50IGlzIHByZXNlbnRpbmcgd2l0aCBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuIAoKIyMgUGxhbgoKVGhlIHBhdGllbnQgd2FzIHByZXNjcmliZWQgdGhlIGZvbGxvd2luZyBtZWRpY2F0aW9uczoKLSBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldAo="}]} +{"resourceType":"DiagnosticReport","id":"08e52e51-ee85-4e52-a192-27d6d817334b","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTAtMDYtMDMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUuIApUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gbWVkaWNhdGlvbiByZWNvbmNpbGlhdGlvbiAocHJvY2VkdXJlKQo="}]} +{"resourceType":"DiagnosticReport","id":"e00fc1e7-2625-4515-8cdf-9b22d41ae0e6","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"},"effectiveDateTime":"2011-03-11T13:15:05-08:00","issued":"2011-03-11T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTEtMDMtMTEKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIHRyaW5lc3NhIDI4IGRheSBwYWNrCg=="}]} +{"resourceType":"DiagnosticReport","id":"c76085ce-ced6-4fb2-8a88-1c62638db794","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTEtMDYtMDkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE5IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjawoKIyBBc3Nlc3NtZW50IGFuZCBQbGFuCgoKIyMgUGxhbgpQYXRpZW50IHdhcyBnaXZlbiB0aGUgZm9sbG93aW5nIGltbXVuaXphdGlvbnM6IGluZmx1ZW56YSwgc2Vhc29uYWwsIGluamVjdGFibGUsIHByZXNlcnZhdGl2ZSBmcmVlLiAK"}]} +{"resourceType":"DiagnosticReport","id":"124c584e-a411-4ef2-8a28-875b40bdffaf","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"},"effectiveDateTime":"2013-02-28T13:15:05-08:00","issued":"2013-02-28T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTMtMDItMjgKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIwIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjawoKIyBBc3Nlc3NtZW50IGFuZCBQbGFuCgoKIyMgUGxhbgoKVGhlIHBhdGllbnQgd2FzIHByZXNjcmliZWQgdGhlIGZvbGxvd2luZyBtZWRpY2F0aW9uczoKLSB5YXogMjggZGF5IHBhY2sK"}]} +{"resourceType":"DiagnosticReport","id":"a5b95e67-0013-4c75-812a-4c1464642fc7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v2-0074","code":"LAB","display":"Laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"58410-2","display":"Complete blood count (hemogram) panel - Blood by Automated count"}],"text":"Complete blood count (hemogram) panel - Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","result":[{"reference":"Observation/4b4e7514-1472-4455-b30d-9632803a4094","display":"Leukocytes [#/volume] in Blood by Automated count"},{"reference":"Observation/53a4daa2-d54e-4ae0-bb11-5a808163f42c","display":"Erythrocytes [#/volume] in Blood by Automated count"},{"reference":"Observation/a6a99aca-ef13-449d-9354-e009d08e8c71","display":"Hemoglobin [Mass/volume] in Blood"},{"reference":"Observation/d60aaa5a-2632-494e-ba83-794ad85127d5","display":"Hematocrit [Volume Fraction] of Blood by Automated count"},{"reference":"Observation/c25c4744-6c81-4a1f-91f8-0f68ddf1f4af","display":"MCV [Entitic volume] by Automated count"},{"reference":"Observation/ff785a7c-4966-4df7-a878-0dcb6de76abb","display":"MCH [Entitic mass] by Automated count"},{"reference":"Observation/c8b433eb-f658-44ae-b7d7-fa0d62cb701a","display":"MCHC [Mass/volume] by Automated count"},{"reference":"Observation/69bcf0d7-7b07-4458-9ff6-788dc2443246","display":"Erythrocyte distribution width [Entitic volume] by Automated count"},{"reference":"Observation/fc5546f2-b5a9-44db-ab27-f04194526e5b","display":"Platelets [#/volume] in Blood by Automated count"},{"reference":"Observation/90029698-691a-430e-bca8-6f54338193fc","display":"Platelet distribution width [Entitic volume] in Blood by Automated count"},{"reference":"Observation/48c316cf-6407-4574-b884-142bb11b0b42","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}]} +{"resourceType":"DiagnosticReport","id":"6c7aed2a-2f9f-4796-85a9-ffa36af2560b","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTQtMDYtMTIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIyIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUsIHRkIChhZHVsdCkgcHJlc2VydmF0aXZlIGZyZWUsIGhlcCBiLCBhZHVsdCwgbWVuaW5nb2NvY2NhbCBtY3Y0cC4gCg=="}]} +{"resourceType":"DiagnosticReport","id":"728336a3-9e9a-4737-8767-b8de22d14385","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"},"effectiveDateTime":"2015-02-18T13:15:05-08:00","issued":"2015-02-18T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTUtMDItMTgKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIyIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW0K"}]} +{"resourceType":"DiagnosticReport","id":"0dfbba83-7fa9-48ab-ab34-a545a161d6e8","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"effectiveDateTime":"2015-04-12T14:15:05-07:00","issued":"2015-04-12T14:15:05.225-07:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTUtMDQtMTIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCjE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggYWN1dGUgYnJvbmNoaXRpcyAoZGlzb3JkZXIpLiAKCiMjIFBsYW4KClRoZSBmb2xsb3dpbmcgcHJvY2VkdXJlcyB3ZXJlIGNvbmR1Y3RlZDoKLSBwbGFpbiBjaGVzdCB4LXJheSAocHJvY2VkdXJlKQpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIGFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0ClRoZSBwYXRpZW50IHdhcyBwbGFjZWQgb24gYSBjYXJlcGxhbjoKLSByZXNwaXJhdG9yeSB0aGVyYXB5Cg=="}]} +{"resourceType":"DiagnosticReport","id":"a359262a-494b-48d5-9eca-52debab2c309","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTUtMDQtMjMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUsIGhlcCBiLCBhZHVsdC4gCg=="}]} +{"resourceType":"DiagnosticReport","id":"ef3508ff-eb22-4144-a6b5-dd745129c055","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"},"effectiveDateTime":"2016-02-13T13:15:05-08:00","issued":"2016-02-13T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTYtMDItMTMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIG5hdGF6aWEgMjggZGF5IHBhY2sK"}]} +{"resourceType":"DiagnosticReport","id":"1a30352d-3766-47dc-b9e6-9dd65f702ab1","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"effectiveDateTime":"2016-10-30T14:15:05-07:00","issued":"2016-10-30T14:15:05.225-07:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTYtMTAtMzAKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI0IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IG5hdGF6aWEgMjggZGF5IHBhY2s7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggZnJhY3R1cmUgb2YgY2xhdmljbGUuIAoKIyMgUGxhbgoKVGhlIGZvbGxvd2luZyBwcm9jZWR1cmVzIHdlcmUgY29uZHVjdGVkOgotIGNsYXZpY2xlIHgtcmF5Ci0gYWRtaXNzaW9uIHRvIG9ydGhvcGVkaWMgZGVwYXJ0bWVudApUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIG1lcGVyaWRpbmUgaHlkcm9jaGxvcmlkZSA1MCBtZyBvcmFsIHRhYmxldAotIGlidXByb2ZlbiAyMDAgbWcgb3JhbCB0YWJsZXQKVGhlIHBhdGllbnQgd2FzIHBsYWNlZCBvbiBhIGNhcmVwbGFuOgotIGZyYWN0dXJlIGNhcmUK"}]} +{"resourceType":"DiagnosticReport","id":"0996a8ad-3dae-4574-97b8-e63bfbed0436","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"},"effectiveDateTime":"2016-12-29T13:15:05-08:00","issued":"2016-12-29T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTYtMTItMjkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI0IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgo="}]} +{"resourceType":"DiagnosticReport","id":"dada411f-d95a-42b6-bea5-987b2da3f0a1","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTctMDYtMTUKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUuIApUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gbWVkaWNhdGlvbiByZWNvbmNpbGlhdGlvbiAocHJvY2VkdXJlKQo="}]} +{"resourceType":"DiagnosticReport","id":"a6d08a29-a694-482d-8df8-36b63c8e8296","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"},"effectiveDateTime":"2018-02-02T13:15:05-08:00","issued":"2018-02-02T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTgtMDItMDIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgo="}]} +{"resourceType":"DiagnosticReport","id":"e1385d55-31f2-4a8b-8356-22869d42d0bd","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"},"effectiveDateTime":"2018-02-09T13:15:05-08:00","issued":"2018-02-09T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTgtMDItMDkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gYmlsYXRlcmFsIHR1YmFsIGxpZ2F0aW9uClRoZSBwYXRpZW50IHdhcyBwbGFjZWQgb24gYSBjYXJlcGxhbjoKLSBtaW5vciBzdXJnZXJ5IGNhcmUgbWFuYWdlbWVudCAocHJvY2VkdXJlKQo="}]} +{"resourceType":"DiagnosticReport","id":"6d6cb707-6e0f-4fe3-a047-8b0b0da44807","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-diagnosticreport-note"]},"status":"final","category":[{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},"effectiveDateTime":"2018-12-11T13:15:05-08:00","issued":"2018-12-11T13:15:05.225-08:00","performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"presentedForm":[{"contentType":"text/plain","data":"CjIwMTgtMTItMTEKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI2IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggc3RyZXB0b2NvY2NhbCBzb3JlIHRocm9hdCAoZGlzb3JkZXIpLiAKCiMjIFBsYW4KClRoZSBmb2xsb3dpbmcgcHJvY2VkdXJlcyB3ZXJlIGNvbmR1Y3RlZDoKLSB0aHJvYXQgY3VsdHVyZSAocHJvY2VkdXJlKQpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0Cg=="}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DocumentReference.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DocumentReference.ndjson new file mode 100644 index 000000000000..c6da76c2a773 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/DocumentReference.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"DocumentReference","id":"a6e558a3-7907-4185-9537-5b3291e05639","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2010-05-19T14:15:05.225-07:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTAtMDUtMTkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCk5vIEFjdGl2ZSBNZWRpY2F0aW9ucy4KCiMgQXNzZXNzbWVudCBhbmQgUGxhbgpQYXRpZW50IGlzIHByZXNlbnRpbmcgd2l0aCBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuIAoKIyMgUGxhbgoKVGhlIHBhdGllbnQgd2FzIHByZXNjcmliZWQgdGhlIGZvbGxvd2luZyBtZWRpY2F0aW9uczoKLSBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldAo="}}],"context":{"encounter":[{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"}],"period":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"}}} +{"resourceType":"DocumentReference","id":"9e6df167-efc0-4b0a-922a-11afba4fe703","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2010-06-03T14:15:05.225-07:00","author":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTAtMDYtMDMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUuIApUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gbWVkaWNhdGlvbiByZWNvbmNpbGlhdGlvbiAocHJvY2VkdXJlKQo="}}],"context":{"encounter":[{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"}],"period":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"}}} +{"resourceType":"DocumentReference","id":"a2a936d2-11dc-46b2-81d2-4f85ba5784ed","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2011-03-11T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTEtMDMtMTEKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE4IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5CiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIHRyaW5lc3NhIDI4IGRheSBwYWNrCg=="}}],"context":{"encounter":[{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"}],"period":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"246b40b8-e424-4be1-b296-de4e7c10e52d","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2011-06-09T14:15:05.225-07:00","author":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTEtMDYtMDkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDE5IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjawoKIyBBc3Nlc3NtZW50IGFuZCBQbGFuCgoKIyMgUGxhbgpQYXRpZW50IHdhcyBnaXZlbiB0aGUgZm9sbG93aW5nIGltbXVuaXphdGlvbnM6IGluZmx1ZW56YSwgc2Vhc29uYWwsIGluamVjdGFibGUsIHByZXNlcnZhdGl2ZSBmcmVlLiAK"}}],"context":{"encounter":[{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"}],"period":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"}}} +{"resourceType":"DocumentReference","id":"99076549-16bc-4843-b6b2-1a912336a5a9","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2013-02-28T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTMtMDItMjgKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIwIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjawoKIyBBc3Nlc3NtZW50IGFuZCBQbGFuCgoKIyMgUGxhbgoKVGhlIHBhdGllbnQgd2FzIHByZXNjcmliZWQgdGhlIGZvbGxvd2luZyBtZWRpY2F0aW9uczoKLSB5YXogMjggZGF5IHBhY2sK"}}],"context":{"encounter":[{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"}],"period":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"f9bba222-4d5f-4b1b-b681-c4ed60c05f89","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2014-06-12T14:15:05.225-07:00","author":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTQtMDYtMTIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIyIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUsIHRkIChhZHVsdCkgcHJlc2VydmF0aXZlIGZyZWUsIGhlcCBiLCBhZHVsdCwgbWVuaW5nb2NvY2NhbCBtY3Y0cC4gCg=="}}],"context":{"encounter":[{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"}],"period":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"}}} +{"resourceType":"DocumentReference","id":"56a84077-f6ac-462b-8c81-40ec2d97b9f6","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2015-02-18T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTUtMDItMTgKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIyIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCnBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW0K"}}],"context":{"encounter":[{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"}],"period":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"f1489031-2951-45c0-8e24-975e7d3eae5d","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2015-04-12T14:15:05.225-07:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTUtMDQtMTIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlcikuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCjE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggYWN1dGUgYnJvbmNoaXRpcyAoZGlzb3JkZXIpLiAKCiMjIFBsYW4KClRoZSBmb2xsb3dpbmcgcHJvY2VkdXJlcyB3ZXJlIGNvbmR1Y3RlZDoKLSBwbGFpbiBjaGVzdCB4LXJheSAocHJvY2VkdXJlKQpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIGFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0ClRoZSBwYXRpZW50IHdhcyBwbGFjZWQgb24gYSBjYXJlcGxhbjoKLSByZXNwaXJhdG9yeSB0aGVyYXB5Cg=="}}],"context":{"encounter":[{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"}],"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"}}} +{"resourceType":"DocumentReference","id":"91115f6d-efb5-4fdc-9ef9-63fe03e21c2a","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2015-04-23T14:15:05.225-07:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTUtMDQtMjMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUsIGhlcCBiLCBhZHVsdC4gCg=="}}],"context":{"encounter":[{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"}],"period":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"}}} +{"resourceType":"DocumentReference","id":"d9054dcc-9a9e-480e-b5dd-3d6a54429c22","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2016-02-13T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTYtMDItMTMKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDIzIHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIG5hdGF6aWEgMjggZGF5IHBhY2sK"}}],"context":{"encounter":[{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"}],"period":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"f77b1642-1b8d-40b1-a1f0-2ff8053d2966","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2016-10-30T14:15:05.225-07:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTYtMTAtMzAKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI0IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKS4KCiMgU29jaWFsIEhpc3RvcnkKUGF0aWVudCBpcyBzaW5nbGUuIFBhdGllbnQgaXMgYW4gYWN0aXZlIHNtb2tlciBhbmQgaXMgYW4gYWxjb2hvbGljLiBQYXRpZW50IGlkZW50aWZpZXMgYXMgaGV0ZXJvc2V4dWFsLgoKUGF0aWVudCBjb21lcyBmcm9tIGEgbWlkZGxlIHNvY2lvZWNvbm9taWMgYmFja2dyb3VuZC4gUGF0aWVudCBoYXMgYSBoaWdoIHNjaG9vbCBlZHVjYXRpb24uIFBhdGllbnQgY3VycmVudGx5IGhhcyBNZWRpY2FpZC4KCiMgQWxsZXJnaWVzCk5vIEtub3duIEFsbGVyZ2llcy4KCiMgTWVkaWNhdGlvbnMKYWNldGFtaW5vcGhlbiAzMjUgbWcgb3JhbCB0YWJsZXQ7IG5hdGF6aWEgMjggZGF5IHBhY2s7IDE2OCBociBldGhpbnlsIGVzdHJhZGlvbCAwLjAwMTQ2IG1nL2hyIC8gbm9yZWxnZXN0cm9taW4gMC4wMDYyNSBtZy9ociB0cmFuc2Rlcm1hbCBzeXN0ZW07IHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0OyB0cmluZXNzYSAyOCBkYXkgcGFjazsgeWF6IDI4IGRheSBwYWNrCgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggZnJhY3R1cmUgb2YgY2xhdmljbGUuIAoKIyMgUGxhbgoKVGhlIGZvbGxvd2luZyBwcm9jZWR1cmVzIHdlcmUgY29uZHVjdGVkOgotIGNsYXZpY2xlIHgtcmF5Ci0gYWRtaXNzaW9uIHRvIG9ydGhvcGVkaWMgZGVwYXJ0bWVudApUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIG1lcGVyaWRpbmUgaHlkcm9jaGxvcmlkZSA1MCBtZyBvcmFsIHRhYmxldAotIGlidXByb2ZlbiAyMDAgbWcgb3JhbCB0YWJsZXQKVGhlIHBhdGllbnQgd2FzIHBsYWNlZCBvbiBhIGNhcmVwbGFuOgotIGZyYWN0dXJlIGNhcmUK"}}],"context":{"encounter":[{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"}],"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"}}} +{"resourceType":"DocumentReference","id":"f30142f1-c8b2-4642-b353-7c1ac79eddd7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2016-12-29T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTYtMTItMjkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI0IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgo="}}],"context":{"encounter":[{"reference":"Encounter/a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"}],"period":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"84fc7c21-e344-431b-922d-ff5ed0beaf48","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2017-06-15T14:15:05.225-07:00","author":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTctMDYtMTUKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuClBhdGllbnQgd2FzIGdpdmVuIHRoZSBmb2xsb3dpbmcgaW1tdW5pemF0aW9uczogaW5mbHVlbnphLCBzZWFzb25hbCwgaW5qZWN0YWJsZSwgcHJlc2VydmF0aXZlIGZyZWUuIApUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gbWVkaWNhdGlvbiByZWNvbmNpbGlhdGlvbiAocHJvY2VkdXJlKQo="}}],"context":{"encounter":[{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"}],"period":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"}}} +{"resourceType":"DocumentReference","id":"b6a6bef9-9602-40d9-bca6-b58315624cbe","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2018-02-02T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTgtMDItMDIKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgo="}}],"context":{"encounter":[{"reference":"Encounter/eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"}],"period":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"}}} +{"resourceType":"DocumentReference","id":"c95bce8c-78b5-4569-807a-0093eec4d721","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"superseded","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2018-02-09T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTgtMDItMDkKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI1IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KCgojIyBQbGFuCgpUaGUgZm9sbG93aW5nIHByb2NlZHVyZXMgd2VyZSBjb25kdWN0ZWQ6Ci0gYmlsYXRlcmFsIHR1YmFsIGxpZ2F0aW9uClRoZSBwYXRpZW50IHdhcyBwbGFjZWQgb24gYSBjYXJlcGxhbjoKLSBtaW5vciBzdXJnZXJ5IGNhcmUgbWFuYWdlbWVudCAocHJvY2VkdXJlKQo="}}],"context":{"encounter":[{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"}],"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"}}} +{"resourceType":"DocumentReference","id":"2cf76961-ab88-4c68-9935-ce10c18faab6","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-documentreference"]},"status":"current","type":{"coding":[{"system":"http://loinc.org","code":"34117-2","display":"History and physical note"},{"system":"http://loinc.org","code":"51847-2","display":"Evaluation+Plan note"}]},"category":[{"coding":[{"system":"http://hl7.org/fhir/us/core/CodeSystem/us-core-documentreference-category","code":"clinical-note","display":"Clinical Note"}]}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"date":"2018-12-11T13:15:05.225-08:00","author":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}],"content":[{"attachment":{"contentType":"text/plain","data":"CjIwMTgtMTItMTEKCiMgQ2hpZWYgQ29tcGxhaW50Ck5vIGNvbXBsYWludHMuCgojIEhpc3Rvcnkgb2YgUHJlc2VudCBJbGxuZXNzCkxvcml0YTIxNyBpcyBhIDI2IHllYXItb2xkIG5vbi1oaXNwYW5pYyB3aGl0ZSBmZW1hbGUuIFBhdGllbnQgaGFzIGEgaGlzdG9yeSBvZiBzdHJlcHRvY29jY2FsIHNvcmUgdGhyb2F0IChkaXNvcmRlciksIGFjdXRlIGJyb25jaGl0aXMgKGRpc29yZGVyKSwgZnJhY3R1cmUgb2YgY2xhdmljbGUuCgojIFNvY2lhbCBIaXN0b3J5ClBhdGllbnQgaXMgc2luZ2xlLiBQYXRpZW50IGlzIGFuIGFjdGl2ZSBzbW9rZXIgYW5kIGlzIGFuIGFsY29ob2xpYy4gUGF0aWVudCBpZGVudGlmaWVzIGFzIGhldGVyb3NleHVhbC4KClBhdGllbnQgY29tZXMgZnJvbSBhIG1pZGRsZSBzb2Npb2Vjb25vbWljIGJhY2tncm91bmQuIFBhdGllbnQgaGFzIGEgaGlnaCBzY2hvb2wgZWR1Y2F0aW9uLiBQYXRpZW50IGN1cnJlbnRseSBoYXMgTWVkaWNhaWQuCgojIEFsbGVyZ2llcwpObyBLbm93biBBbGxlcmdpZXMuCgojIE1lZGljYXRpb25zCmFjZXRhbWlub3BoZW4gMzI1IG1nIG9yYWwgdGFibGV0OyBuYXRhemlhIDI4IGRheSBwYWNrOyAxNjggaHIgZXRoaW55bCBlc3RyYWRpb2wgMC4wMDE0NiBtZy9ociAvIG5vcmVsZ2VzdHJvbWluIDAuMDA2MjUgbWcvaHIgdHJhbnNkZXJtYWwgc3lzdGVtOyBwZW5pY2lsbGluIHYgcG90YXNzaXVtIDUwMCBtZyBvcmFsIHRhYmxldDsgaWJ1cHJvZmVuIDIwMCBtZyBvcmFsIHRhYmxldDsgdHJpbmVzc2EgMjggZGF5IHBhY2s7IHlheiAyOCBkYXkgcGFjazsgbWVwZXJpZGluZSBoeWRyb2NobG9yaWRlIDUwIG1nIG9yYWwgdGFibGV0CgojIEFzc2Vzc21lbnQgYW5kIFBsYW4KUGF0aWVudCBpcyBwcmVzZW50aW5nIHdpdGggc3RyZXB0b2NvY2NhbCBzb3JlIHRocm9hdCAoZGlzb3JkZXIpLiAKCiMjIFBsYW4KClRoZSBmb2xsb3dpbmcgcHJvY2VkdXJlcyB3ZXJlIGNvbmR1Y3RlZDoKLSB0aHJvYXQgY3VsdHVyZSAocHJvY2VkdXJlKQpUaGUgcGF0aWVudCB3YXMgcHJlc2NyaWJlZCB0aGUgZm9sbG93aW5nIG1lZGljYXRpb25zOgotIHBlbmljaWxsaW4gdiBwb3Rhc3NpdW0gNTAwIG1nIG9yYWwgdGFibGV0Cg=="}}],"context":{"encounter":[{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"}],"period":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"}}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Encounter.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Encounter.ndjson new file mode 100644 index 000000000000..d973abf02d66 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Encounter.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"Encounter","id":"84fa1324-9220-4cf6-bf35-fce580dbb7a8","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"84fa1324-9220-4cf6-bf35-fce580dbb7a8"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"7c8b9bd7-0172-47af-bd21-6af5b0eacf08","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"7c8b9bd7-0172-47af-bd21-6af5b0eacf08"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}}],"period":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"location":[{"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}}],"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}} +{"resourceType":"Encounter","id":"2dd9110c-d725-45cf-827f-bafec9b3543f","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"2dd9110c-d725-45cf-827f-bafec9b3543f"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"f60a41fa-6fbf-4fc4-b565-488d6197e2f4","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"f60a41fa-6fbf-4fc4-b565-488d6197e2f4"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}}],"period":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"location":[{"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}}],"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}} +{"resourceType":"Encounter","id":"9ff3cb23-2995-4a1a-92ac-ff0a36dbec59","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"8627419c-d90e-43ce-9035-74578faa9e15","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"8627419c-d90e-43ce-9035-74578faa9e15"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}}],"period":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"location":[{"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}}],"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}} +{"resourceType":"Encounter","id":"ea1c03fb-e18d-4b93-8fea-79b98c095cbc","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"ea1c03fb-e18d-4b93-8fea-79b98c095cbc"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"12b98110-3df6-40cb-bb70-87ea01d4aa31","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"12b98110-3df6-40cb-bb70-87ea01d4aa31"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}]}],"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"47e0d3ec-0263-4fba-916e-43bf4abe166e","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"47e0d3ec-0263-4fba-916e-43bf4abe166e"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}],"text":"Encounter for check up (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"1d49703f-4c67-4a71-bb97-849b8915b718","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"1d49703f-4c67-4a71-bb97-849b8915b718"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"e32f7ba3-c22a-4222-8b53-58a3118450f7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"e32f7ba3-c22a-4222-8b53-58a3118450f7"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"EMER"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}],"text":"Encounter for 'check-up'"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}]}],"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"14349254-a133-4fac-8055-2571bc83f059","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"14349254-a133-4fac-8055-2571bc83f059"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"}}],"period":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"location":[{"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}}],"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}} +{"resourceType":"Encounter","id":"eac7e5d9-b34a-4353-81b2-5a1a40df2d7d","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"498dc8b8-db3d-4064-87eb-4011786324d5","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"498dc8b8-db3d-4064-87eb-4011786324d5"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"IMP"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}],"text":"Admission to surgical department"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Encounter","id":"dd86ac8c-7740-41f3-8dfc-7ac26e90a448","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-encounter"]},"identifier":[{"use":"official","system":"https://github.com/synthetichealth/synthea","value":"dd86ac8c-7740-41f3-8dfc-7ac26e90a448"}],"status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c","display":"Ms. Lorita217 Nolan344"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}}],"period":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"reasonCode":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"location":[{"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ExplanationOfBenefit.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ExplanationOfBenefit.ndjson new file mode 100644 index 000000000000..6a785b678a13 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ExplanationOfBenefit.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"ExplanationOfBenefit","id":"648291cb-e6ec-478a-894e-b653e7ae929b","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"a7b49375-f326-462c-91a6-fb68c3d7c359"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-05-19T14:45:05-07:00","end":"2011-05-19T14:45:05-07:00"},"created":"2010-05-19T14:45:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/a7b49375-f326-462c-91a6-fb68c3d7c359"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7"},"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-diagnosistype","code":"principal"}]}]}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"servicedPeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"}]},{"sequence":2,"diagnosisSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"servicedPeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]}}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"2400fc5c-fb9e-4703-a8b6-42dd2c50f5e5","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"d4f07a36-012e-4170-8757-0c9bfbdfa575"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-06-03T14:45:05-07:00","end":"2011-06-03T14:45:05-07:00"},"created":"2010-06-03T14:45:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"claim":{"reference":"Claim/d4f07a36-012e-4170-8757-0c9bfbdfa575"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"servicedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"encounter":[{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"}]},{"sequence":2,"informationSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"servicedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"servicedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":504.83,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":100.96600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":403.86400000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":504.83,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":504.83,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":516.2800000000001,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"96d4c306-c1c8-4619-a79c-45c9d495beec","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"6556aaa0-bf9f-40e7-bc54-1c3ceec565c7"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-03-11T13:30:05-08:00","end":"2012-03-11T13:30:05-07:00"},"created":"2011-03-11T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/6556aaa0-bf9f-40e7-bc54-1c3ceec565c7"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"servicedPeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"b8290c3e-815f-4e76-89a0-d245fc1f34f5","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"d8f548d1-bf09-4fb3-a0d0-5454feee2396"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-06-09T14:30:05-07:00","end":"2012-06-09T14:30:05-07:00"},"created":"2011-06-09T14:30:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"claim":{"reference":"Claim/d8f548d1-bf09-4fb3-a0d0-5454feee2396"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"servicedPeriod":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"encounter":[{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"}]},{"sequence":2,"informationSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"servicedPeriod":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":112.41600000000001,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"d5608f86-b442-4f45-b320-c509103e46f1","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"171190a5-7d0d-45a4-99ac-0c519b6dd270"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2013-02-28T13:30:05-08:00","end":"2014-02-28T13:30:05-08:00"},"created":"2013-02-28T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/171190a5-7d0d-45a4-99ac-0c519b6dd270"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"servicedPeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"1d89ba47-1aa8-4ac4-9605-fa831c02d973","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"50387d60-9d96-475d-9719-7b9cb30071ca"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2014-06-12T14:30:05-07:00","end":"2015-06-12T14:30:05-07:00"},"created":"2014-06-12T14:30:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"claim":{"reference":"Claim/50387d60-9d96-475d-9719-7b9cb30071ca"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"servicedPeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"encounter":[{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"}]},{"sequence":2,"informationSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"servicedPeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":3,"informationSequence":[2],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"}],"text":"Td (adult) preservative free"},"servicedPeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":4,"informationSequence":[3],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"servicedPeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":5,"informationSequence":[4],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"}],"text":"meningococcal MCV4P"},"servicedPeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":449.66400000000004,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"38b3e5e7-6c5d-42b2-8e32-13481e4ca696","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"43604c79-af2f-4ea3-a871-a6efeb148686"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-02-18T13:30:05-08:00","end":"2016-02-18T13:30:05-08:00"},"created":"2015-02-18T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/43604c79-af2f-4ea3-a871-a6efeb148686"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"servicedPeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"b09fc58f-f447-4193-9557-fd08c4ff0ff0","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"5c4943b7-52fe-43cf-b800-200585136db8"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-12T14:41:05-07:00","end":"2016-04-12T14:41:05-07:00"},"created":"2015-04-12T14:41:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/5c4943b7-52fe-43cf-b800-200585136db8"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9"},"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-diagnosistype","code":"principal"}]}]}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"servicedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"}]},{"sequence":2,"diagnosisSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"},"servicedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]}},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"}],"text":"Plain chest X-ray (procedure)"},"servicedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"net":{"value":6535.58,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":1307.116,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":5228.464,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":6535.58,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":6535.58,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":5228.464,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"f534820d-1611-43ea-9fd0-5850a2f34633","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"99fc6fc4-98f4-415a-a094-f22df454163f"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-23T14:30:05-07:00","end":"2016-04-23T14:30:05-07:00"},"created":"2015-04-23T14:30:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/99fc6fc4-98f4-415a-a094-f22df454163f"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}],"text":"Encounter for check up (procedure)"},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"}]},{"sequence":2,"informationSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":3,"informationSequence":[2],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":224.83200000000002,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"ebcc7838-65a0-46cd-bb54-a00d8d224280","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"4a41154a-06aa-4bfc-94ce-b96f28f11b80"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-02-13T13:30:05-08:00","end":"2017-02-13T13:30:05-08:00"},"created":"2016-02-13T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/4a41154a-06aa-4bfc-94ce-b96f28f11b80"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"servicedPeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"1fb7d685-cf96-4660-8c12-240da51c3640","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"07ae31e4-8480-4293-b506-5e9b32e7491b"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T16:45:05-07:00","end":"2017-10-30T16:45:05-07:00"},"created":"2016-10-30T16:45:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/07ae31e4-8480-4293-b506-5e9b32e7491b"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/115098be-c26e-47d0-9cad-be9df75d7042"},"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-diagnosistype","code":"principal"}]}]}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"20","display":"Urgent Care Facility"}]},"encounter":[{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"}]},{"sequence":2,"diagnosisSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"20","display":"Urgent Care Facility"}]}},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"}],"text":"Clavicle X-ray"},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"20","display":"Urgent Care Facility"}]},"net":{"value":516.65,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":103.33,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":413.32,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":516.65,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":516.65,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":4,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"}],"text":"Admission to orthopedic department"},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"20","display":"Urgent Care Facility"}]},"net":{"value":516.65,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":103.33,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":413.32,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":516.65,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":516.65,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":826.64,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"cf6fa314-eae2-4989-b585-a41249329ee8","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"b60311e1-94d9-473c-bf5f-ea06db1dad15"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-12-29T13:30:05-08:00","end":"2017-12-29T13:30:05-08:00"},"created":"2016-12-29T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/b60311e1-94d9-473c-bf5f-ea06db1dad15"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}],"text":"Encounter for 'check-up'"},"servicedPeriod":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"315a9c6b-8ea5-4b7b-8943-5e0de6175218","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"performer":[{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"b52c9dfa-56bb-4523-9ace-38ad614ee107"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2017-06-15T14:45:05-07:00","end":"2018-06-15T14:45:05-07:00"},"created":"2017-06-15T14:45:05-07:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"},"claim":{"reference":"Claim/b52c9dfa-56bb-4523-9ace-38ad614ee107"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"},"servicedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"encounter":[{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"}]},{"sequence":2,"informationSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"servicedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":140.52,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"servicedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"19","display":"Off Campus-Outpatient Hospital"}]},"net":{"value":482.02,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":96.404,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":385.616,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":482.02,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":482.02,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":498.032,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"08f6e8ec-9017-4919-aeb3-7b659e24e5ba","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"084d3b6a-e1d4-4e25-b90b-c970fbc20eb7"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-02-02T13:30:05-08:00","end":"2019-02-02T13:30:05-08:00"},"created":"2018-02-02T13:30:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/084d3b6a-e1d4-4e25-b90b-c970fbc20eb7"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"},"servicedPeriod":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":0.0,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"dc9c5273-41ad-42ef-8dde-1c21768c0149","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"cf98de5a-9843-43c6-84c6-e6e703ffa457"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-02-10T15:15:05-08:00","end":"2019-02-10T15:15:05-08:00"},"created":"2018-02-10T15:15:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/cf98de5a-9843-43c6-84c6-e6e703ffa457"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}],"text":"Admission to surgical department"},"servicedPeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"}]},{"sequence":2,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"}],"text":"Bilateral tubal ligation"},"servicedPeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"net":{"value":9364.49,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":1872.8980000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":7491.592000000001,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":9364.49,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":9364.49,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":7491.592000000001,"currency":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"f43fb2ac-d805-4e61-97f4-415cf14814f1","contained":[{"resourceType":"ServiceRequest","id":"referral","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"performer":[{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}]},{"resourceType":"Coverage","id":"coverage","status":"active","type":{"text":"Medicaid"},"beneficiary":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"payor":[{"display":"Medicaid"}]}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"15be99a3-ce23-43fd-a32a-fbda483a1224"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claim-type","code":"institutional"}]},"use":"claim","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-12-11T13:45:05-08:00","end":"2019-12-11T13:45:05-08:00"},"created":"2018-12-11T13:45:05-08:00","insurer":{"display":"Medicaid"},"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"referral":{"reference":"#referral"},"facility":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"claim":{"reference":"Claim/15be99a3-ce23-43fd-a32a-fbda483a1224"},"outcome":"complete","careTeam":[{"sequence":1,"provider":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"role":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/9c0af3aa-42c9-43a6-8677-ae5fa3f34922"},"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-diagnosistype","code":"principal"}]}]}],"insurance":[{"focal":true,"coverage":{"reference":"#coverage","display":"Medicaid"}}],"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"},"servicedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"}]},{"sequence":2,"diagnosisSequence":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"servicedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]}},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"productOrService":{"coding":[{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"}],"text":"Throat culture (procedure)"},"servicedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/ex-serviceplace","code":"21","display":"Inpatient Hospital"}]},"net":{"value":1958.61,"currency":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":391.722,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":1566.888,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":1958.61,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":1958.61,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"currency":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"total":[{"category":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/adjudication","code":"submitted","display":"Submitted Amount"}],"text":"Submitted Amount"},"amount":{"value":129.16,"currency":"USD"}}],"payment":{"amount":{"value":1566.888,"currency":"USD"}}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ImagingStudy.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ImagingStudy.ndjson new file mode 100644 index 000000000000..ca85bc9f5ad0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/ImagingStudy.ndjson @@ -0,0 +1 @@ +{"resourceType":"ImagingStudy","id":"91afdb24-acac-411d-995e-0d36187ea211","identifier":[{"use":"official","system":"urn:ietf:rfc:3986","value":"urn:oid:1.2.840.99999999.22502730.1589831196459"}],"status":"available","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"started":"2016-10-30T14:15:05-07:00","numberOfSeries":1,"numberOfInstances":1,"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"series":[{"uid":"1.2.840.99999999.1.65971777.1589831196459","number":1,"modality":{"system":"http://dicom.nema.org/resources/ontology/DCM","code":"DX","display":"Digital Radiography"},"numberOfInstances":1,"bodySite":{"system":"http://snomed.info/sct","code":"51299004","display":"Clavicle"},"started":"2016-10-30T14:15:05-07:00","instance":[{"uid":"1.2.840.99999999.1.1.48010377.1589831196459","sopClass":{"system":"urn:ietf:rfc:3986","code":"1.2.840.10008.5.1.4.1.1.1.1"},"number":1,"title":"Image of clavicle"}]}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Immunization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Immunization.ndjson new file mode 100644 index 000000000000..cf15293f2474 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Immunization.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"Immunization","id":"f28a9afd-a143-45db-8aeb-082b0df22426","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"occurrenceDateTime":"2010-06-03T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"e755857d-8ce3-42b6-81f0-d5b52524fae5","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"occurrenceDateTime":"2011-06-09T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"490a18a8-30e4-4a65-9a8c-0993671cf186","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"occurrenceDateTime":"2014-06-12T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"e451e36e-dc92-4fd9-a924-1a83a04da939","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"}],"text":"Td (adult) preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"occurrenceDateTime":"2014-06-12T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"673cd32a-bf63-4c7a-9e57-1b92d9fb15e9","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"occurrenceDateTime":"2014-06-12T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"41daff0e-fb2b-4216-8065-1481d7c962ed","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"}],"text":"meningococcal MCV4P"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"occurrenceDateTime":"2014-06-12T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Immunization","id":"aebc3fce-9411-4246-87ee-17a65f3f5f9a","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"occurrenceDateTime":"2015-04-23T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Immunization","id":"ab327672-36ba-4980-83a8-c4a1525de444","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"occurrenceDateTime":"2015-04-23T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Immunization","id":"bc61721c-4053-4a81-b1e5-b64194e1efa1","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-immunization"]},"status":"completed","vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"occurrenceDateTime":"2017-06-15T14:15:05-07:00","primarySource":true,"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Location.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Location.ndjson new file mode 100644 index 000000000000..dc8773c211f9 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Location.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Location","id":"e39647c2-31b8-4e0c-b383-0994cd478ca0","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-location"]},"status":"active","name":"METROWEST MEDICAL CENTER","telecom":[{"system":"phone","value":"5083831000"}],"address":{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"},"position":{"longitude":-71.436196,"latitude":42.307905},"managingOrganization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Location","id":"7a41edc1-c724-4bf9-b850-c78f9b1d387d","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-location"]},"status":"active","name":"PCP68975","telecom":[{"system":"phone","value":"508-881-4368"}],"address":{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"},"position":{"longitude":-71.473526,"latitude":42.257754999999996},"managingOrganization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/MedicationRequest.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/MedicationRequest.ndjson new file mode 100644 index 000000000000..2e45acdbad98 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/MedicationRequest.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"MedicationRequest","id":"8a21a17d-b0b3-48b2-8427-5eb9d1138628","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"},"authoredOn":"2010-05-19T14:15:05-07:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"reasonReference":[{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7"}]} +{"resourceType":"MedicationRequest","id":"c9954767-c8b7-45e0-8b07-099351dcd9a0","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"751905","display":"Trinessa 28 Day Pack"}],"text":"Trinessa 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"},"authoredOn":"2011-03-11T13:15:05-08:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}} +{"resourceType":"MedicationRequest","id":"d10f9ae0-ffef-4a9b-8021-74a6cd6b1ad5","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"748856","display":"Yaz 28 Day Pack"}],"text":"Yaz 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"},"authoredOn":"2013-02-28T13:15:05-08:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}} +{"resourceType":"MedicationRequest","id":"c316061f-f9ad-4a30-80cf-c488b64b1112","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"1534809","display":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"}],"text":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"},"authoredOn":"2015-02-18T13:15:05-08:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}} +{"resourceType":"MedicationRequest","id":"4b2bf1a7-6801-4ed0-85d2-336106458fdc","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"313782","display":"Acetaminophen 325 MG Oral Tablet"}],"text":"Acetaminophen 325 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"authoredOn":"2015-04-12T14:15:05-07:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"reasonReference":[{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9"}]} +{"resourceType":"MedicationRequest","id":"1df5e2a6-88ef-423c-ae94-5972e6ed7d36","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"978950","display":"Natazia 28 Day Pack"}],"text":"Natazia 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"},"authoredOn":"2016-02-13T13:15:05-08:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"}} +{"resourceType":"MedicationRequest","id":"7202acd4-409c-4078-87bb-125f9ffe4bad","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"861467","display":"Meperidine Hydrochloride 50 MG Oral Tablet"}],"text":"Meperidine Hydrochloride 50 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"authoredOn":"2016-10-30T14:15:05-07:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"dosageInstruction":[{"sequence":1,"timing":{"repeat":{"frequency":1,"period":4.0,"periodUnit":"h"}},"asNeededBoolean":false,"doseAndRate":[{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/dose-rate-type","code":"ordered","display":"Ordered"}]},"doseQuantity":{"value":1.0}}]}]} +{"resourceType":"MedicationRequest","id":"edfc50e7-5843-4cbe-b4d2-b8a0c5b6bcfe","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"310965","display":"Ibuprofen 200 MG Oral Tablet"}],"text":"Ibuprofen 200 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"authoredOn":"2016-10-30T14:15:05-07:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"dosageInstruction":[{"sequence":1,"asNeededBoolean":true}]} +{"resourceType":"MedicationRequest","id":"86aaf462-e2c8-4cdd-8db1-5edb78f15ea1","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-medicationrequest"]},"status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},"authoredOn":"2018-12-11T13:15:05-08:00","requester":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"reasonReference":[{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7"},{"reference":"Condition/9c0af3aa-42c9-43a6-8677-ae5fa3f34922"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Observation.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Observation.ndjson new file mode 100644 index 000000000000..e81bd862bb17 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Observation.ndjson @@ -0,0 +1,54 @@ +{"resourceType":"Observation","id":"df86f58b-b6bb-43cf-9a62-623c1f5bc51c","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyheight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":153.6,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"06a121ab-df4b-4336-833b-e36bd44f5387","status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":1,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"379d9202-c0c8-46ee-8a27-edac176ff215","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyweight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":55.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"9c8009e2-8704-4150-b33b-5b43eaea568c","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bmi","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":23.41,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"63584063-b54c-4363-a1b2-21ab1ec06d60","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/pediatric-bmi-for-age"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":71.788,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"dbe27317-6771-4cfd-a250-d6260ad4da83","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":72,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":1.2E+2,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"0df769ad-ca90-4020-b214-e30665499e31","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/heartrate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"0f7a3b4c-9919-4642-85f1-37538e1a6018","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/resprate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"470a3225-f3e1-47f6-bd23-ac09b6f3b0c2","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-smokingstatus"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"ff702f77-3aeb-44a9-91bb-e663930ba12d","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyheight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":153.7,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"66c84e05-25e6-426c-9d5e-7bef76a79a7e","status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"5ebce2f1-3136-4318-9ffb-acfe4ecf0a92","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyweight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":62.1,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"bcce2383-307d-420c-a14a-f4e2d0969093","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bmi","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":26.3,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"5f3c6fbb-024c-4119-94fb-f6c603bd2834","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/pediatric-bmi-for-age"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":85.491,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"a962c9cc-5c29-4e3f-91df-369aa539a2fe","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":88,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":121,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"c8306162-fcde-4a60-aa71-7875f95c8c9a","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/heartrate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":93,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"81faaa81-ea0c-4c7c-aaec-cb05a354d6a9","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/resprate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"759df675-5dfc-4e52-83f6-55bbc7f099c4","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-smokingstatus"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"20c3ca50-9aaa-4b11-b202-66976092df3f","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyheight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"b1d50c74-8e57-4086-88fa-74e62885be4e","status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"992a5862-97d3-4b37-8100-c6a906b0c819","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyweight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":69.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"ea97bfc2-c5b8-4b59-b1e4-4b72a94269f9","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bmi","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":29.25,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"7a97a0ca-2d18-427c-afda-68c6140b7359","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":87,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":122,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"ae5fc42b-2535-47a8-a180-2494487a8281","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/heartrate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":9E+1,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"8e9f3656-5bea-4473-b8b3-414f6d8c0c49","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/resprate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":16,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"4b4e7514-1472-4455-b30d-9632803a4094","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"6690-2","display":"Leukocytes [#/volume] in Blood by Automated count"}],"text":"Leukocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":7.2857,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"53a4daa2-d54e-4ae0-bb11-5a808163f42c","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"789-8","display":"Erythrocytes [#/volume] in Blood by Automated count"}],"text":"Erythrocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":4.7109,"unit":"10*6/uL","system":"http://unitsofmeasure.org","code":"10*6/uL"}} +{"resourceType":"Observation","id":"a6a99aca-ef13-449d-9354-e009d08e8c71","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"718-7","display":"Hemoglobin [Mass/volume] in Blood"}],"text":"Hemoglobin [Mass/volume] in Blood"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":15.964,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"d60aaa5a-2632-494e-ba83-794ad85127d5","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"4544-3","display":"Hematocrit [Volume Fraction] of Blood by Automated count"}],"text":"Hematocrit [Volume Fraction] of Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":45.934,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"c25c4744-6c81-4a1f-91f8-0f68ddf1f4af","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"787-2","display":"MCV [Entitic volume] by Automated count"}],"text":"MCV [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":94.793,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"ff785a7c-4966-4df7-a878-0dcb6de76abb","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"785-6","display":"MCH [Entitic mass] by Automated count"}],"text":"MCH [Entitic mass] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":28.937,"unit":"pg","system":"http://unitsofmeasure.org","code":"pg"}} +{"resourceType":"Observation","id":"c8b433eb-f658-44ae-b7d7-fa0d62cb701a","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"786-4","display":"MCHC [Mass/volume] by Automated count"}],"text":"MCHC [Mass/volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":33.479,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"69bcf0d7-7b07-4458-9ff6-788dc2443246","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"21000-5","display":"Erythrocyte distribution width [Entitic volume] by Automated count"}],"text":"Erythrocyte distribution width [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":43.303,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"fc5546f2-b5a9-44db-ab27-f04194526e5b","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"777-3","display":"Platelets [#/volume] in Blood by Automated count"}],"text":"Platelets [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":173.89,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"90029698-691a-430e-bca8-6f54338193fc","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"32207-3","display":"Platelet distribution width [Entitic volume] in Blood by Automated count"}],"text":"Platelet distribution width [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":413.95,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"48c316cf-6407-4574-b884-142bb11b0b42","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-observation-lab"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"32623-1","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}],"text":"Platelet mean volume [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":10.07,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"9ea6c443-bb4a-4e8d-a810-fd57e1cb0607","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-smokingstatus"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"a494d6f8-d2ef-49dd-9673-f759e885dd33","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyheight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"40b93af4-f801-4966-b980-8dece84be05c","status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"dacd8c8a-6cd8-4c95-82d5-a89688fd42e2","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyweight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":70.3,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"8b03de48-f251-4056-b351-2cef4cde7071","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bmi","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":29.73,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"5c432044-547b-4217-9d30-7f6dc62c34f5","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":79,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":129,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"40ef3a2c-1458-44dc-a197-39c8ccfd84f5","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/heartrate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":9E+1,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"10b3fc73-9295-498b-a958-9f026898f480","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/resprate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":15,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"03d9292b-015f-4b16-bc25-a3c928a9e4f7","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-smokingstatus"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"5490ec27-04dd-4e77-8e95-13334038599d","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyheight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"2c78b705-6228-43d0-a13a-e44dc653327d","status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":0,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"69a5d27f-352d-45e1-af40-6883c90a63f9","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodyweight","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66.8,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"77253638-9ca7-47f6-a7a7-40fde67593a7","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bmi","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":28.22,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"beba7f58-c49e-41e4-8f6d-5bf11107ef36","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":69,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":111,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"d4e540aa-9747-41ad-b1b6-04c972af0b38","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/heartrate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"5fb102db-15c6-4b87-a97d-44264a7ed7fc","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/resprate","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":12,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"24893400-afed-44fe-8faf-5ac5cfe8dd34","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-smokingstatus"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"3989f7ef-48ee-4a8a-a2a6-422f293cad4e","meta":{"profile":["http://hl7.org/fhir/StructureDefinition/bodytemp","http://hl7.org/fhir/StructureDefinition/vitalsigns"]},"status":"final","category":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8310-5","display":"Body temperature"},{"system":"http://loinc.org","code":"8331-1","display":"Oral temperature"}],"text":"Body temperature"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},"effectiveDateTime":"2018-12-11T13:15:05-08:00","issued":"2018-12-11T13:15:05.225-08:00","valueQuantity":{"value":39.024,"unit":"Cel","system":"http://unitsofmeasure.org","code":"Cel"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Organization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Organization.ndjson new file mode 100644 index 000000000000..e1036faf02c4 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Organization.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Organization","id":"465de31f-3098-365c-af70-48a071e1f5aa","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization"]},"identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"465de31f-3098-365c-af70-48a071e1f5aa"}],"active":true,"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/organization-type","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"}],"name":"METROWEST MEDICAL CENTER","telecom":[{"system":"phone","value":"5083831000"}],"address":[{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}]} +{"resourceType":"Organization","id":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization"]},"identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}],"active":true,"type":[{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/organization-type","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"}],"name":"PCP68975","telecom":[{"system":"phone","value":"508-881-4368"}],"address":[{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Patient.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Patient.ndjson new file mode 100644 index 000000000000..e86812afb434 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Patient.ndjson @@ -0,0 +1 @@ +{"resourceType":"Patient","id":"1416dec1-f4b1-4b48-b7f4-650e8f67499c","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-patient"]},"text":{"status":"generated","div":"
    Generated by Synthea.Version identifier: v2.5.0-265-gbd5a00e8\n . Person seed: 6732543839779682504 Population seed: 1589831189867
    "},"extension":[{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-race","extension":[{"url":"ombCategory","valueCoding":{"system":"urn:oid:2.16.840.1.113883.6.238","code":"2106-3","display":"White"}},{"url":"text","valueString":"White"}]},{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity","extension":[{"url":"ombCategory","valueCoding":{"system":"urn:oid:2.16.840.1.113883.6.238","code":"2186-5","display":"Not Hispanic or Latino"}},{"url":"text","valueString":"Not Hispanic or Latino"}]},{"url":"http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName","valueString":"Leanna255 Predovic534"},{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex","valueCode":"F"},{"url":"http://hl7.org/fhir/StructureDefinition/patient-birthPlace","valueAddress":{"city":"Southbridge","state":"Massachusetts","country":"US"}},{"url":"http://synthetichealth.github.io/synthea/disability-adjusted-life-years","valueDecimal":0.0},{"url":"http://synthetichealth.github.io/synthea/quality-adjusted-life-years","valueDecimal":27.0}],"identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v2-0203","code":"MR","display":"Medical Record Number"}],"text":"Medical Record Number"},"system":"http://hospital.smarthealthit.org","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v2-0203","code":"SS","display":"Social Security Number"}],"text":"Social Security Number"},"system":"http://hl7.org/fhir/sid/us-ssn","value":"999-54-3579"},{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v2-0203","code":"DL","display":"Driver's License"}],"text":"Driver's License"},"system":"urn:oid:2.16.840.1.113883.4.3.25","value":"S99972984"},{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v2-0203","code":"PPN","display":"Passport Number"}],"text":"Passport Number"},"system":"http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber","value":"X39621585X"}],"name":[{"use":"official","family":"Nolan344","given":["Lorita217"],"prefix":["Ms."]}],"telecom":[{"system":"phone","value":"555-817-6998","use":"home"}],"gender":"female","birthDate":"1992-04-09","address":[{"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/geolocation","extension":[{"url":"latitude","valueDecimal":42.27693107900605},{"url":"longitude","valueDecimal":-71.45741653702677}]}],"line":["330 Sawayn Parade"],"city":"Framingham","state":"MA","country":"US"}],"maritalStatus":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/v3-MaritalStatus","code":"S","display":"S"}],"text":"S"},"multipleBirthBoolean":false,"communication":[{"language":{"coding":[{"system":"urn:ietf:bcp:47","code":"en-US","display":"English"}],"text":"English"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Practitioner.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Practitioner.ndjson new file mode 100644 index 000000000000..1ff54077e908 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Practitioner.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Practitioner","id":"c16820ae-2954-32d4-863c-e9ceb741154c","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner"]},"identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"530"}],"active":true,"name":[{"family":"Murphy561","given":["Mari763"],"prefix":["Dr."]}],"telecom":[{"extension":[{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct","valueBoolean":true}],"system":"email","value":"Mari763.Murphy561@example.com","use":"work"}],"address":[{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}],"gender":"female"} +{"resourceType":"Practitioner","id":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner"]},"identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"35750"}],"active":true,"name":[{"family":"Hilpert278","given":["Cathryn51"],"prefix":["Dr."]}],"telecom":[{"extension":[{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct","valueBoolean":true}],"system":"email","value":"Cathryn51.Hilpert278@example.com","use":"work"}],"address":[{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}],"gender":"female"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/PractitionerRole.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/PractitionerRole.ndjson new file mode 100644 index 000000000000..72bd745ca8d0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/PractitionerRole.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"PractitionerRole","id":"47501a3e-8eaa-4432-9b47-750769e4bf56","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole"]},"practitioner":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c","display":"Dr. Mari763 Murphy561"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa","display":"METROWEST MEDICAL CENTER"},"code":[{"coding":[{"system":"http://nucc.org/provider-taxonomy","code":"208D00000X","display":"General Practice"}],"text":"General Practice"}],"specialty":[{"coding":[{"system":"http://nucc.org/provider-taxonomy","code":"208D00000X","display":"General Practice"}],"text":"General Practice"}],"location":[{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}],"telecom":[{"system":"phone","value":"5083831000"}]} +{"resourceType":"PractitionerRole","id":"af8e91ae-aedf-419e-bd06-cfcf8a1c612f","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole"]},"practitioner":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","display":"Dr. Cathryn51 Hilpert278"},"organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"},"code":[{"coding":[{"system":"http://nucc.org/provider-taxonomy","code":"208D00000X","display":"General Practice"}],"text":"General Practice"}],"specialty":[{"coding":[{"system":"http://nucc.org/provider-taxonomy","code":"208D00000X","display":"General Practice"}],"text":"General Practice"}],"location":[{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}],"telecom":[{"system":"phone","value":"508-881-4368"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Procedure.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Procedure.ndjson new file mode 100644 index 000000000000..6bda489a5287 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Procedure.ndjson @@ -0,0 +1,7 @@ +{"resourceType":"Procedure","id":"eba3e367-30ea-418d-ae97-e91506dea85f","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},"performedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:30:05-07:00"},"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Procedure","id":"78d69a34-6c80-4a6c-abbb-0accca0a17a5","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"}],"text":"Plain chest X-ray (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},"performedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:26:05-07:00"},"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"reasonReference":[{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9","display":"Acute bronchitis (disorder)"}]} +{"resourceType":"Procedure","id":"e614b129-913f-42ac-bcc3-266db50af307","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"}],"text":"Clavicle X-ray"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T14:45:05-07:00"},"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Procedure","id":"cf1cfcd7-ca27-4bf1-b8f0-14e31c69824e","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"}],"text":"Admission to orthopedic department"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T15:15:05-07:00"},"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"reasonReference":[{"reference":"Condition/115098be-c26e-47d0-9cad-be9df75d7042","display":"Fracture of clavicle"}]} +{"resourceType":"Procedure","id":"ea19efb7-6346-45e9-aa8b-3195cc0b8aee","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},"performedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:30:05-07:00"},"location":{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d","display":"PCP68975"}} +{"resourceType":"Procedure","id":"91112614-fd2f-46cb-ad49-96111084ef1c","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"}],"text":"Bilateral tubal ligation"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"},"performedPeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-09T15:15:05-08:00"},"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"}} +{"resourceType":"Procedure","id":"1b7f052f-0bb7-422e-86c9-0c8f6af49fe2","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-procedure"]},"status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"}],"text":"Throat culture (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},"performedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:30:05-08:00"},"location":{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0","display":"METROWEST MEDICAL CENTER"},"reasonReference":[{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7","display":"Streptococcal sore throat (disorder)"},{"reference":"Condition/9c0af3aa-42c9-43a6-8677-ae5fa3f34922","display":"Streptococcal sore throat (disorder)"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Provenance.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Provenance.ndjson new file mode 100644 index 000000000000..736f152ede70 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/Provenance.ndjson @@ -0,0 +1 @@ +{"resourceType":"Provenance","id":"71dac48d-2e1b-4fce-8615-0d1c27a0be2d","meta":{"profile":["http://hl7.org/fhir/us/core/StructureDefinition/us-core-provenance"]},"target":[{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"reference":"Location/e39647c2-31b8-4e0c-b383-0994cd478ca0"},{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},{"reference":"PractitionerRole/47501a3e-8eaa-4432-9b47-750769e4bf56"},{"reference":"Encounter/84fa1324-9220-4cf6-bf35-fce580dbb7a8"},{"reference":"Condition/4336ae40-60ee-4148-9271-a372cb9600a7"},{"reference":"MedicationRequest/8a21a17d-b0b3-48b2-8427-5eb9d1138628"},{"reference":"Claim/e24412c9-d798-4a45-8d29-98027d1eaa5d"},{"reference":"DiagnosticReport/c31327f8-f2ec-4b26-a8bd-40aff65cc085"},{"reference":"DocumentReference/a6e558a3-7907-4185-9537-5b3291e05639"},{"reference":"Claim/a7b49375-f326-462c-91a6-fb68c3d7c359"},{"reference":"ExplanationOfBenefit/648291cb-e6ec-478a-894e-b653e7ae929b"},{"reference":"Location/7a41edc1-c724-4bf9-b850-c78f9b1d387d"},{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"},{"reference":"PractitionerRole/af8e91ae-aedf-419e-bd06-cfcf8a1c612f"},{"reference":"Encounter/7c8b9bd7-0172-47af-bd21-6af5b0eacf08"},{"reference":"Observation/df86f58b-b6bb-43cf-9a62-623c1f5bc51c"},{"reference":"Observation/06a121ab-df4b-4336-833b-e36bd44f5387"},{"reference":"Observation/379d9202-c0c8-46ee-8a27-edac176ff215"},{"reference":"Observation/9c8009e2-8704-4150-b33b-5b43eaea568c"},{"reference":"Observation/63584063-b54c-4363-a1b2-21ab1ec06d60"},{"reference":"Observation/dbe27317-6771-4cfd-a250-d6260ad4da83"},{"reference":"Observation/0df769ad-ca90-4020-b214-e30665499e31"},{"reference":"Observation/0f7a3b4c-9919-4642-85f1-37538e1a6018"},{"reference":"Observation/470a3225-f3e1-47f6-bd23-ac09b6f3b0c2"},{"reference":"Procedure/eba3e367-30ea-418d-ae97-e91506dea85f"},{"reference":"Immunization/f28a9afd-a143-45db-8aeb-082b0df22426"},{"reference":"DiagnosticReport/08e52e51-ee85-4e52-a192-27d6d817334b"},{"reference":"DocumentReference/9e6df167-efc0-4b0a-922a-11afba4fe703"},{"reference":"Claim/d4f07a36-012e-4170-8757-0c9bfbdfa575"},{"reference":"ExplanationOfBenefit/2400fc5c-fb9e-4703-a8b6-42dd2c50f5e5"},{"reference":"Encounter/2dd9110c-d725-45cf-827f-bafec9b3543f"},{"reference":"MedicationRequest/c9954767-c8b7-45e0-8b07-099351dcd9a0"},{"reference":"Claim/e4ae0e04-c0b5-47c0-9f1c-1a4cefe48a97"},{"reference":"DiagnosticReport/e00fc1e7-2625-4515-8cdf-9b22d41ae0e6"},{"reference":"DocumentReference/a2a936d2-11dc-46b2-81d2-4f85ba5784ed"},{"reference":"Claim/6556aaa0-bf9f-40e7-bc54-1c3ceec565c7"},{"reference":"ExplanationOfBenefit/96d4c306-c1c8-4619-a79c-45c9d495beec"},{"reference":"Encounter/f60a41fa-6fbf-4fc4-b565-488d6197e2f4"},{"reference":"Observation/ff702f77-3aeb-44a9-91bb-e663930ba12d"},{"reference":"Observation/66c84e05-25e6-426c-9d5e-7bef76a79a7e"},{"reference":"Observation/5ebce2f1-3136-4318-9ffb-acfe4ecf0a92"},{"reference":"Observation/bcce2383-307d-420c-a14a-f4e2d0969093"},{"reference":"Observation/5f3c6fbb-024c-4119-94fb-f6c603bd2834"},{"reference":"Observation/a962c9cc-5c29-4e3f-91df-369aa539a2fe"},{"reference":"Observation/c8306162-fcde-4a60-aa71-7875f95c8c9a"},{"reference":"Observation/81faaa81-ea0c-4c7c-aaec-cb05a354d6a9"},{"reference":"Observation/759df675-5dfc-4e52-83f6-55bbc7f099c4"},{"reference":"Immunization/e755857d-8ce3-42b6-81f0-d5b52524fae5"},{"reference":"DiagnosticReport/c76085ce-ced6-4fb2-8a88-1c62638db794"},{"reference":"DocumentReference/246b40b8-e424-4be1-b296-de4e7c10e52d"},{"reference":"Claim/d8f548d1-bf09-4fb3-a0d0-5454feee2396"},{"reference":"ExplanationOfBenefit/b8290c3e-815f-4e76-89a0-d245fc1f34f5"},{"reference":"Encounter/9ff3cb23-2995-4a1a-92ac-ff0a36dbec59"},{"reference":"MedicationRequest/d10f9ae0-ffef-4a9b-8021-74a6cd6b1ad5"},{"reference":"Claim/2d2de9d8-9f05-4979-b19f-62d0733cbe9b"},{"reference":"DiagnosticReport/124c584e-a411-4ef2-8a28-875b40bdffaf"},{"reference":"DocumentReference/99076549-16bc-4843-b6b2-1a912336a5a9"},{"reference":"Claim/171190a5-7d0d-45a4-99ac-0c519b6dd270"},{"reference":"ExplanationOfBenefit/d5608f86-b442-4f45-b320-c509103e46f1"},{"reference":"Encounter/8627419c-d90e-43ce-9035-74578faa9e15"},{"reference":"Observation/20c3ca50-9aaa-4b11-b202-66976092df3f"},{"reference":"Observation/b1d50c74-8e57-4086-88fa-74e62885be4e"},{"reference":"Observation/992a5862-97d3-4b37-8100-c6a906b0c819"},{"reference":"Observation/ea97bfc2-c5b8-4b59-b1e4-4b72a94269f9"},{"reference":"Observation/7a97a0ca-2d18-427c-afda-68c6140b7359"},{"reference":"Observation/ae5fc42b-2535-47a8-a180-2494487a8281"},{"reference":"Observation/8e9f3656-5bea-4473-b8b3-414f6d8c0c49"},{"reference":"Observation/4b4e7514-1472-4455-b30d-9632803a4094"},{"reference":"Observation/53a4daa2-d54e-4ae0-bb11-5a808163f42c"},{"reference":"Observation/a6a99aca-ef13-449d-9354-e009d08e8c71"},{"reference":"Observation/d60aaa5a-2632-494e-ba83-794ad85127d5"},{"reference":"Observation/c25c4744-6c81-4a1f-91f8-0f68ddf1f4af"},{"reference":"Observation/ff785a7c-4966-4df7-a878-0dcb6de76abb"},{"reference":"Observation/c8b433eb-f658-44ae-b7d7-fa0d62cb701a"},{"reference":"Observation/69bcf0d7-7b07-4458-9ff6-788dc2443246"},{"reference":"Observation/fc5546f2-b5a9-44db-ab27-f04194526e5b"},{"reference":"Observation/90029698-691a-430e-bca8-6f54338193fc"},{"reference":"Observation/48c316cf-6407-4574-b884-142bb11b0b42"},{"reference":"Observation/9ea6c443-bb4a-4e8d-a810-fd57e1cb0607"},{"reference":"Immunization/490a18a8-30e4-4a65-9a8c-0993671cf186"},{"reference":"Immunization/e451e36e-dc92-4fd9-a924-1a83a04da939"},{"reference":"Immunization/673cd32a-bf63-4c7a-9e57-1b92d9fb15e9"},{"reference":"Immunization/41daff0e-fb2b-4216-8065-1481d7c962ed"},{"reference":"DiagnosticReport/a5b95e67-0013-4c75-812a-4c1464642fc7"},{"reference":"DiagnosticReport/6c7aed2a-2f9f-4796-85a9-ffa36af2560b"},{"reference":"DocumentReference/f9bba222-4d5f-4b1b-b681-c4ed60c05f89"},{"reference":"Claim/50387d60-9d96-475d-9719-7b9cb30071ca"},{"reference":"ExplanationOfBenefit/1d89ba47-1aa8-4ac4-9605-fa831c02d973"},{"reference":"Encounter/ea1c03fb-e18d-4b93-8fea-79b98c095cbc"},{"reference":"MedicationRequest/c316061f-f9ad-4a30-80cf-c488b64b1112"},{"reference":"Claim/39917e96-c35b-4c72-9abf-222dacb4d19e"},{"reference":"DiagnosticReport/728336a3-9e9a-4737-8767-b8de22d14385"},{"reference":"DocumentReference/56a84077-f6ac-462b-8c81-40ec2d97b9f6"},{"reference":"Claim/43604c79-af2f-4ea3-a871-a6efeb148686"},{"reference":"ExplanationOfBenefit/38b3e5e7-6c5d-42b2-8e32-13481e4ca696"},{"reference":"Encounter/12b98110-3df6-40cb-bb70-87ea01d4aa31"},{"reference":"Condition/c8d97153-17c7-4a2b-bafd-052ff8f30eb9"},{"reference":"Procedure/78d69a34-6c80-4a6c-abbb-0accca0a17a5"},{"reference":"MedicationRequest/4b2bf1a7-6801-4ed0-85d2-336106458fdc"},{"reference":"Claim/9e1c8ca0-e622-46c2-ae92-5b32f9ebfff0"},{"reference":"CareTeam/111c8459-549e-4a2b-ab70-27f3b2408eac"},{"reference":"CarePlan/32841402-d5d8-4615-af90-473bf6f417fa"},{"reference":"DiagnosticReport/0dfbba83-7fa9-48ab-ab34-a545a161d6e8"},{"reference":"DocumentReference/f1489031-2951-45c0-8e24-975e7d3eae5d"},{"reference":"Claim/5c4943b7-52fe-43cf-b800-200585136db8"},{"reference":"ExplanationOfBenefit/b09fc58f-f447-4193-9557-fd08c4ff0ff0"},{"reference":"Encounter/47e0d3ec-0263-4fba-916e-43bf4abe166e"},{"reference":"Observation/a494d6f8-d2ef-49dd-9673-f759e885dd33"},{"reference":"Observation/40b93af4-f801-4966-b980-8dece84be05c"},{"reference":"Observation/dacd8c8a-6cd8-4c95-82d5-a89688fd42e2"},{"reference":"Observation/8b03de48-f251-4056-b351-2cef4cde7071"},{"reference":"Observation/5c432044-547b-4217-9d30-7f6dc62c34f5"},{"reference":"Observation/40ef3a2c-1458-44dc-a197-39c8ccfd84f5"},{"reference":"Observation/10b3fc73-9295-498b-a958-9f026898f480"},{"reference":"Observation/03d9292b-015f-4b16-bc25-a3c928a9e4f7"},{"reference":"Immunization/aebc3fce-9411-4246-87ee-17a65f3f5f9a"},{"reference":"Immunization/ab327672-36ba-4980-83a8-c4a1525de444"},{"reference":"DiagnosticReport/a359262a-494b-48d5-9eca-52debab2c309"},{"reference":"DocumentReference/91115f6d-efb5-4fdc-9ef9-63fe03e21c2a"},{"reference":"Claim/99fc6fc4-98f4-415a-a094-f22df454163f"},{"reference":"ExplanationOfBenefit/f534820d-1611-43ea-9fd0-5850a2f34633"},{"reference":"Encounter/1d49703f-4c67-4a71-bb97-849b8915b718"},{"reference":"MedicationRequest/1df5e2a6-88ef-423c-ae94-5972e6ed7d36"},{"reference":"Claim/8c0e3992-24e7-4c44-9c50-3f0ea1431726"},{"reference":"DiagnosticReport/ef3508ff-eb22-4144-a6b5-dd745129c055"},{"reference":"DocumentReference/d9054dcc-9a9e-480e-b5dd-3d6a54429c22"},{"reference":"Claim/4a41154a-06aa-4bfc-94ce-b96f28f11b80"},{"reference":"ExplanationOfBenefit/ebcc7838-65a0-46cd-bb54-a00d8d224280"},{"reference":"Encounter/e32f7ba3-c22a-4222-8b53-58a3118450f7"},{"reference":"Condition/115098be-c26e-47d0-9cad-be9df75d7042"},{"reference":"Procedure/e614b129-913f-42ac-bcc3-266db50af307"},{"reference":"Procedure/cf1cfcd7-ca27-4bf1-b8f0-14e31c69824e"},{"reference":"MedicationRequest/7202acd4-409c-4078-87bb-125f9ffe4bad"},{"reference":"Claim/80484d29-eded-431d-bf9b-8a917089a77e"},{"reference":"MedicationRequest/edfc50e7-5843-4cbe-b4d2-b8a0c5b6bcfe"},{"reference":"Claim/41a00b32-22ff-4fd3-a388-296e4cc77dfe"},{"reference":"CareTeam/31d5cf9a-c745-4087-8042-6a9b9babaef9"},{"reference":"CarePlan/ac2893f1-9c90-437d-852f-4cc1a0155d39"},{"reference":"ImagingStudy/91afdb24-acac-411d-995e-0d36187ea211"},{"reference":"DiagnosticReport/1a30352d-3766-47dc-b9e6-9dd65f702ab1"},{"reference":"DocumentReference/f77b1642-1b8d-40b1-a1f0-2ff8053d2966"},{"reference":"Claim/07ae31e4-8480-4293-b506-5e9b32e7491b"},{"reference":"ExplanationOfBenefit/1fb7d685-cf96-4660-8c12-240da51c3640"},{"reference":"Encounter/a6db95e4-0a32-4a9b-a1b6-9f43b6e5e9f7"},{"reference":"DiagnosticReport/0996a8ad-3dae-4574-97b8-e63bfbed0436"},{"reference":"DocumentReference/f30142f1-c8b2-4642-b353-7c1ac79eddd7"},{"reference":"Claim/b60311e1-94d9-473c-bf5f-ea06db1dad15"},{"reference":"ExplanationOfBenefit/cf6fa314-eae2-4989-b585-a41249329ee8"},{"reference":"Encounter/14349254-a133-4fac-8055-2571bc83f059"},{"reference":"Observation/5490ec27-04dd-4e77-8e95-13334038599d"},{"reference":"Observation/2c78b705-6228-43d0-a13a-e44dc653327d"},{"reference":"Observation/69a5d27f-352d-45e1-af40-6883c90a63f9"},{"reference":"Observation/77253638-9ca7-47f6-a7a7-40fde67593a7"},{"reference":"Observation/beba7f58-c49e-41e4-8f6d-5bf11107ef36"},{"reference":"Observation/d4e540aa-9747-41ad-b1b6-04c972af0b38"},{"reference":"Observation/5fb102db-15c6-4b87-a97d-44264a7ed7fc"},{"reference":"Observation/24893400-afed-44fe-8faf-5ac5cfe8dd34"},{"reference":"Procedure/ea19efb7-6346-45e9-aa8b-3195cc0b8aee"},{"reference":"Immunization/bc61721c-4053-4a81-b1e5-b64194e1efa1"},{"reference":"DiagnosticReport/dada411f-d95a-42b6-bea5-987b2da3f0a1"},{"reference":"DocumentReference/84fc7c21-e344-431b-922d-ff5ed0beaf48"},{"reference":"Claim/b52c9dfa-56bb-4523-9ace-38ad614ee107"},{"reference":"ExplanationOfBenefit/315a9c6b-8ea5-4b7b-8943-5e0de6175218"},{"reference":"Encounter/eac7e5d9-b34a-4353-81b2-5a1a40df2d7d"},{"reference":"DiagnosticReport/a6d08a29-a694-482d-8df8-36b63c8e8296"},{"reference":"DocumentReference/b6a6bef9-9602-40d9-bca6-b58315624cbe"},{"reference":"Claim/084d3b6a-e1d4-4e25-b90b-c970fbc20eb7"},{"reference":"ExplanationOfBenefit/08f6e8ec-9017-4919-aeb3-7b659e24e5ba"},{"reference":"Encounter/498dc8b8-db3d-4064-87eb-4011786324d5"},{"reference":"Procedure/91112614-fd2f-46cb-ad49-96111084ef1c"},{"reference":"CareTeam/e1ab345a-81bd-461e-921d-9eae42c47f8a"},{"reference":"CarePlan/5a1fb9d1-310b-4fe3-b7c3-6370529400b8"},{"reference":"DiagnosticReport/e1385d55-31f2-4a8b-8356-22869d42d0bd"},{"reference":"DocumentReference/c95bce8c-78b5-4569-807a-0093eec4d721"},{"reference":"Claim/cf98de5a-9843-43c6-84c6-e6e703ffa457"},{"reference":"ExplanationOfBenefit/dc9c5273-41ad-42ef-8dde-1c21768c0149"},{"reference":"Encounter/dd86ac8c-7740-41f3-8dfc-7ac26e90a448"},{"reference":"Condition/9c0af3aa-42c9-43a6-8677-ae5fa3f34922"},{"reference":"Observation/3989f7ef-48ee-4a8a-a2a6-422f293cad4e"},{"reference":"Procedure/1b7f052f-0bb7-422e-86c9-0c8f6af49fe2"},{"reference":"MedicationRequest/86aaf462-e2c8-4cdd-8db1-5edb78f15ea1"},{"reference":"Claim/bfc5accc-20be-4556-b76a-e076fc4b2278"},{"reference":"DiagnosticReport/6d6cb707-6e0f-4fe3-a047-8b0b0da44807"},{"reference":"DocumentReference/2cf76961-ab88-4c68-9935-ce10c18faab6"},{"reference":"Claim/15be99a3-ce23-43fd-a32a-fbda483a1224"},{"reference":"ExplanationOfBenefit/f43fb2ac-d805-4e61-97f4-415cf14814f1"}],"recorded":"2020-05-21T14:15:05.225-07:00","agent":[{"type":{"coding":[{"system":"http://terminology.hl7.org/CodeSystem/provenance-participant-type","code":"author","display":"Author"}],"text":"Author"},"who":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","display":"PCP68975"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json new file mode 100644 index 000000000000..b80501554861 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json @@ -0,0 +1,318 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:5cd31fbf-76ba-4e8c-84dd-6c72071c98cf", + "resource": { + "resourceType": "Location", + "id": "5cd31fbf-76ba-4e8c-84dd-6c72071c98cf", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" + ] + }, + "status": "active", + "name": "METROWEST MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5083831000" + } + ], + "address": { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + }, + "position": { + "longitude": -71.436196, + "latitude": 42.307905 + }, + "managingOrganization": { + "reference": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", + "display": "METROWEST MEDICAL CENTER" + } + } + }, + { + "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", + "resource": { + "resourceType": "Organization", + "id": "465de31f-3098-365c-af70-48a071e1f5aa", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "465de31f-3098-365c-af70-48a071e1f5aa" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "METROWEST MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5083831000" + } + ], + "address": [ + { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:274697c5-2d9d-412f-b825-194f88241aec", + "resource": { + "resourceType": "Location", + "id": "274697c5-2d9d-412f-b825-194f88241aec", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" + ] + }, + "status": "active", + "name": "PCP68975", + "telecom": [ + { + "system": "phone", + "value": "508-881-4368" + } + ], + "address": { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + }, + "position": { + "longitude": -71.473526, + "latitude": 42.257754999999996 + }, + "managingOrganization": { + "reference": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "display": "PCP68975" + } + } + }, + { + "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "resource": { + "resourceType": "Organization", + "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 23 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 5 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 17 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP68975", + "telecom": [ + { + "system": "phone", + "value": "508-881-4368" + } + ], + "address": [ + { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:31352f93-39d4-447d-bb77-8cac27d5c9d0", + "resource": { + "resourceType": "Location", + "id": "31352f93-39d4-447d-bb77-8cac27d5c9d0", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" + ] + }, + "status": "active", + "name": "WALTHAM URGENT CARE", + "telecom": [ + { + "system": "phone", + "value": "617-243-5591" + } + ], + "address": { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + }, + "position": { + "longitude": -71.2490152, + "latitude": 42.3700513 + }, + "managingOrganization": { + "reference": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "display": "WALTHAM URGENT CARE" + } + } + }, + { + "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "resource": { + "resourceType": "Organization", + "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "WALTHAM URGENT CARE", + "telecom": [ + { + "system": "phone", + "value": "617-243-5591" + } + ], + "address": [ + { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ] + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json new file mode 100644 index 000000000000..90fcfcc493a2 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json @@ -0,0 +1,354 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", + "resource": { + "resourceType": "Practitioner", + "id": "c16820ae-2954-32d4-863c-e9ceb741154c", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "530" + } + ], + "active": true, + "name": [ + { + "family": "Murphy561", + "given": [ + "Mari763" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", + "valueBoolean": true + } + ], + "system": "email", + "value": "Mari763.Murphy561@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:70dbe417-06dc-4c57-93ce-84209ac38ff5", + "resource": { + "resourceType": "PractitionerRole", + "id": "70dbe417-06dc-4c57-93ce-84209ac38ff5", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" + ] + }, + "practitioner": { + "reference": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", + "display": "Dr. Mari763 Murphy561" + }, + "organization": { + "reference": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", + "display": "METROWEST MEDICAL CENTER" + }, + "code": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "specialty": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "location": [ + { + "display": "METROWEST MEDICAL CENTER" + } + ], + "telecom": [ + { + "system": "phone", + "value": "5083831000" + } + ] + } + }, + { + "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "resource": { + "resourceType": "Practitioner", + "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "35750" + } + ], + "active": true, + "name": [ + { + "family": "Hilpert278", + "given": [ + "Cathryn51" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", + "valueBoolean": true + } + ], + "system": "email", + "value": "Cathryn51.Hilpert278@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:d163bdcf-8294-4320-85b8-86e0680dc34a", + "resource": { + "resourceType": "PractitionerRole", + "id": "d163bdcf-8294-4320-85b8-86e0680dc34a", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" + ] + }, + "practitioner": { + "reference": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "display": "Dr. Cathryn51 Hilpert278" + }, + "organization": { + "reference": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "display": "PCP68975" + }, + "code": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "specialty": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "location": [ + { + "display": "PCP68975" + } + ], + "telecom": [ + { + "system": "phone", + "value": "508-881-4368" + } + ] + } + }, + { + "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", + "resource": { + "resourceType": "Practitioner", + "id": "861aa935-d89f-3b41-861b-b791fa112253", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" + ] + }, + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93440" + } + ], + "active": true, + "name": [ + { + "family": "Reynolds644", + "given": [ + "Salena230" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", + "valueBoolean": true + } + ], + "system": "email", + "value": "Salena230.Reynolds644@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:689adf0e-16fd-4546-9764-69d26ef64777", + "resource": { + "resourceType": "PractitionerRole", + "id": "689adf0e-16fd-4546-9764-69d26ef64777", + "meta": { + "profile": [ + "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" + ] + }, + "practitioner": { + "reference": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", + "display": "Dr. Salena230 Reynolds644" + }, + "organization": { + "reference": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "display": "WALTHAM URGENT CARE" + }, + "code": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "specialty": [ + { + "coding": [ + { + "system": "http://nucc.org/provider-taxonomy", + "code": "208D00000X", + "display": "General Practice" + } + ], + "text": "General Practice" + } + ], + "location": [ + { + "display": "WALTHAM URGENT CARE" + } + ], + "telecom": [ + { + "system": "phone", + "value": "617-243-5591" + } + ] + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/CarePlan.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/CarePlan.ndjson new file mode 100644 index 000000000000..99dec1a97ca0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/CarePlan.ndjson @@ -0,0 +1,3 @@ +{"resourceType":"CarePlan","id":"e1162c3b-86cf-4097-a33d-34bb9a93cfb9","status":"completed","intent":"order","category":[{"coding":[{"system":"http://snomed.info/sct","code":"53950000","display":"Respiratory therapy"}],"text":"Respiratory therapy"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"},"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-19T14:15:05-07:00"},"addresses":[{"reference":"Condition/5953e3df-ed6c-4509-8d56-04c745817bd0"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"304510005","display":"Recommendation to avoid exercise"}],"text":"Recommendation to avoid exercise"},"status":"completed"}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"371605008","display":"Deep breathing and coughing exercises"}],"text":"Deep breathing and coughing exercises"},"status":"completed"}}]} +{"resourceType":"CarePlan","id":"04330c15-061f-494c-9d2f-4e42eca9e099","status":"completed","intent":"order","category":[{"coding":[{"system":"http://snomed.info/sct","code":"385691007","display":"Fracture care"}],"text":"Fracture care"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-12-29T13:15:05-08:00"},"addresses":[{"reference":"Condition/0aa16b8a-bce6-4362-8312-879d7ccf29f8"}],"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed"}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"408580007","display":"Physical activity target light exercise"}],"text":"Physical activity target light exercise"},"status":"completed"}}]} +{"resourceType":"CarePlan","id":"28ffafe4-9be5-4eb5-b663-c22d10c82cf0","status":"completed","intent":"order","category":[{"coding":[{"system":"http://snomed.info/sct","code":"737471002","display":"Minor surgery care management (procedure)"}],"text":"Minor surgery care management (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/89c75588-d3a0-4ff8-8202-665ff4349fa7"},"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-23T13:15:05-08:00"},"activity":[{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"183051005","display":"Recommendation to rest"}],"text":"Recommendation to rest"},"status":"completed"}},{"detail":{"code":{"coding":[{"system":"http://snomed.info/sct","code":"243077000","display":"Recommendation to limit sexual activity"}],"text":"Recommendation to limit sexual activity"},"status":"completed"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Claim.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Claim.ndjson new file mode 100644 index 000000000000..7c94153bf3d3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Claim.ndjson @@ -0,0 +1,25 @@ +{"resourceType":"Claim","id":"a88729f5-c37e-4949-85a7-a4a69ffedecb","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/df4ce22a-1ae5-429f-8d76-5a503a3921ab"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/7b569404-f475-45d9-94d3-b4e31fece687"}]}],"total":{"value":10.45,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"462a2021-f2c0-45cc-85ec-3338664ebd9e","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/f8fe7590-249c-4b93-8bf0-275f2feb6b81"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/7b569404-f475-45d9-94d3-b4e31fece687"}]},{"sequence":2,"diagnosisLinkId":[1]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"bf23b373-969f-4bc7-89a2-b91d321f54d1","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"information":[{"sequence":1,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/293b214e-ae9b-4053-8ed1-becbe613de18"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/63702bbb-e289-4a80-9f1a-3ef5a7d15019"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"}]},{"sequence":2,"informationLinkId":[1],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"procedureLinkId":[1],"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"430193006"}]},"net":{"value":504.83,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"94f416b6-2750-401d-af06-0f3391518cf2","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/8c7b34ec-760b-412b-bfd4-ec560b2ac998"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/ebe5ce5b-57bb-4368-93d6-0b7a0da073a1"}]}],"total":{"value":31.0,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"2bf9a87f-6c7e-4f56-b63f-4a5675faefb7","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/ebe5ce5b-57bb-4368-93d6-0b7a0da073a1"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"0dd76242-6c52-45ce-9e13-84e635c25f4a","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"information":[{"sequence":1,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/c04ae3a8-1678-404b-8fb7-2d249ba82839"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"}]},{"sequence":2,"informationLinkId":[1],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"527cff9c-e3d1-462e-8fcc-1058cf6955f1","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/546161c7-df48-41ae-a25f-2dbbf39729be"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/6245224b-6d15-448a-8e56-db606acf21d8"}]}],"total":{"value":26.21,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"7f84d563-6fb1-4339-8432-766672b1737b","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/6245224b-6d15-448a-8e56-db606acf21d8"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"9e3e2f01-530d-40b1-9795-eba32a97158b","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"information":[{"sequence":1,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/60ab5005-b572-498c-9e32-a5acdc26e3f8"}},{"sequence":2,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/dc26c48b-cdd3-4384-a0f2-02d25682a02d"}},{"sequence":3,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/52ebf7c5-8af2-4f91-9e9a-8b54de12f857"}},{"sequence":4,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/ded2b0fb-56ed-4245-85c8-0b2a02ff4b53"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"}]},{"sequence":2,"informationLinkId":[1],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"informationLinkId":[2],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":4,"informationLinkId":[3],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":5,"informationLinkId":[4],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"384aade3-08c4-4723-b9f0-029501cb4453","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/c9970c0c-236e-4431-bfe9-c08cdd622411"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/7b2e1b32-79cc-412f-b8e7-97997da45089"}]}],"total":{"value":45.32,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"0623afa7-5753-450b-9124-c93df6a8a4f7","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/7b2e1b32-79cc-412f-b8e7-97997da45089"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"f1a3cb88-268b-4308-aaa1-b7cc8f429d43","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/6cb4c447-aa9d-4bfc-89f3-e424dab1fd2a"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"}]}],"total":{"value":5.32,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"abafc9ba-edc7-4eff-98fc-e41d8785c27b","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/5953e3df-ed6c-4509-8d56-04c745817bd0"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/bbedb163-5b9d-4865-bd9f-d59f7a9c21ad"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"}]},{"sequence":2,"diagnosisLinkId":[1]},{"sequence":3,"procedureLinkId":[1],"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"399208008"}]},"net":{"value":6535.58,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"9bd3bd28-4e88-4d8a-8510-ea791953455b","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"information":[{"sequence":1,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/8ea94a08-14b2-4b18-9f70-7d5f8ae729f7"}},{"sequence":2,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/8ca4d799-58a3-4930-a35c-c6f5a1555173"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"}]},{"sequence":2,"informationLinkId":[1],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"informationLinkId":[2],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"73babc33-4afd-401f-a275-3d1e98c6007e","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/e42328cf-bb27-470f-9cff-a9dcb8fd7485"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/96baca61-8639-457b-bd48-fa32511b5985"}]}],"total":{"value":31.25,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"36c0e892-2f4c-4387-8587-7fe004a6bafc","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/96baca61-8639-457b-bd48-fa32511b5985"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"aecbb5b4-1438-4bc3-a845-dd64a9868765","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/b8c23eb0-d9b6-4413-8293-62c13b47e2ab"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"}]}],"total":{"value":113.81,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"e069f335-0774-4bf9-b4e0-821ab7c5908e","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/22ba50dd-b723-4771-9247-2b25e566da68"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"}]}],"total":{"value":17.35,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"d69ba5dd-6da7-4d7b-8cce-63b1ed0c5134","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/0aa16b8a-bce6-4362-8312-879d7ccf29f8"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/565fedff-5363-4938-9b49-1a2b5b004e15"}},{"sequence":2,"procedureReference":{"reference":"Procedure/faae5e23-a03a-450a-ad12-50ca56eaa0e1"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"}]},{"sequence":2,"diagnosisLinkId":[1]},{"sequence":3,"procedureLinkId":[1],"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"168594001"}]},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":4,"procedureLinkId":[2],"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"305428000"}]},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"4cb0502f-3892-41f6-ac59-49d31dc28193","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/522f6ab4-fd6e-42d4-b7c1-c25c99e35d3a"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"f7183ce1-f163-40b3-8ed8-0bcff268937f","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"organization":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"},"information":[{"sequence":1,"category":{"coding":[{"system":"http://hl7.org/fhir/claiminformationcategory","code":"info"}]},"valueReference":{"reference":"Immunization/69152acf-13e8-4265-bd5f-7a2df80a08f2"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/2b4a074e-e62c-4208-9ae5-be371b9208d3"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"}]},{"sequence":2,"informationLinkId":[1],"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"sequence":3,"procedureLinkId":[1],"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"430193006"}]},"net":{"value":482.02,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"e72892e7-dbe8-45ce-b6bf-4d0952c618e3","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/6fb5a1fb-c93c-4648-bb60-db870634bbf2"}]}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"8d5d5189-0c2c-4bf9-930a-7bc245908702","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/28e04a1a-c56a-4aca-a144-ec986f09f1aa"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/89c75588-d3a0-4ff8-8202-665ff4349fa7"}]},{"sequence":2,"procedureLinkId":[1],"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"287664005"}]},"net":{"value":9364.49,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"6332d509-9603-41af-905d-f1961edd19d3","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"prescription":{"reference":"MedicationRequest/f313632b-f3da-4507-a3de-954c156bd799"},"item":[{"sequence":1,"encounter":[{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"}]}],"total":{"value":16.98,"system":"urn:iso:std:iso:4217","code":"USD"}} +{"resourceType":"Claim","id":"d220ef38-fff4-45e8-94e0-da6716f84ff6","status":"active","use":"complete","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"organization":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"},"diagnosis":[{"sequence":1,"diagnosisReference":{"reference":"Condition/53dac70f-c5cc-4880-a96e-fdb157db8e3b"}}],"procedure":[{"sequence":1,"procedureReference":{"reference":"Procedure/17543a9f-3947-4a3c-af28-21391ee2674a"}}],"item":[{"sequence":1,"encounter":[{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"}]},{"sequence":2,"diagnosisLinkId":[1]},{"sequence":3,"procedureLinkId":[1],"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"117015009"}]},"net":{"value":1958.61,"system":"urn:iso:std:iso:4217","code":"USD"}}],"total":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Condition.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Condition.ndjson new file mode 100644 index 000000000000..ce06594b71d7 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Condition.ndjson @@ -0,0 +1,4 @@ +{"resourceType":"Condition","id":"f8fe7590-249c-4b93-8bf0-275f2feb6b81","clinicalStatus":"resolved","verificationStatus":"confirmed","code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/7b569404-f475-45d9-94d3-b4e31fece687"},"onsetDateTime":"2010-05-19T14:15:05-07:00","abatementDateTime":"2010-05-26T14:15:05-07:00","assertedDate":"2010-05-19T14:15:05-07:00"} +{"resourceType":"Condition","id":"5953e3df-ed6c-4509-8d56-04c745817bd0","clinicalStatus":"resolved","verificationStatus":"confirmed","code":{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}],"text":"Acute bronchitis (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"},"onsetDateTime":"2015-04-12T14:15:05-07:00","abatementDateTime":"2015-04-19T14:15:05-07:00","assertedDate":"2015-04-12T14:15:05-07:00"} +{"resourceType":"Condition","id":"0aa16b8a-bce6-4362-8312-879d7ccf29f8","clinicalStatus":"resolved","verificationStatus":"confirmed","code":{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}],"text":"Fracture of clavicle"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"onsetDateTime":"2016-10-30T14:15:05-07:00","abatementDateTime":"2016-12-29T13:15:05-08:00","assertedDate":"2016-10-30T14:15:05-07:00"} +{"resourceType":"Condition","id":"53dac70f-c5cc-4880-a96e-fdb157db8e3b","clinicalStatus":"resolved","verificationStatus":"confirmed","code":{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}],"text":"Streptococcal sore throat (disorder)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"},"onsetDateTime":"2018-12-11T13:15:05-08:00","abatementDateTime":"2018-12-19T13:15:05-08:00","assertedDate":"2018-12-11T13:15:05-08:00"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/DiagnosticReport.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/DiagnosticReport.ndjson new file mode 100644 index 000000000000..826daad9fed0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/DiagnosticReport.ndjson @@ -0,0 +1 @@ +{"resourceType":"DiagnosticReport","id":"89228305-514c-4b1a-8000-cda6db77eccc","status":"final","code":{"coding":[{"system":"http://loinc.org","code":"58410-2","display":"Complete blood count (hemogram) panel - Blood by Automated count"}],"text":"Complete blood count (hemogram) panel - Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","result":[{"reference":"Observation/a71330cb-bdf0-4383-9411-0e3372da9577","display":"Leukocytes [#/volume] in Blood by Automated count"},{"reference":"Observation/f74b75ea-1ad6-4418-ae8b-2bd62e875e21","display":"Erythrocytes [#/volume] in Blood by Automated count"},{"reference":"Observation/573fb355-b0bf-4af5-98d8-ead5389eb3cc","display":"Hemoglobin [Mass/volume] in Blood"},{"reference":"Observation/d750ec32-b61a-40da-b58d-fff11de5bf27","display":"Hematocrit [Volume Fraction] of Blood by Automated count"},{"reference":"Observation/ab86a392-7cca-4625-97d4-421aeaa14382","display":"MCV [Entitic volume] by Automated count"},{"reference":"Observation/0bb6c8de-bf30-43bd-9ae7-b56bc3c74914","display":"MCH [Entitic mass] by Automated count"},{"reference":"Observation/ff40e33c-caee-4181-b71f-cf9a12f7a24f","display":"MCHC [Mass/volume] by Automated count"},{"reference":"Observation/2e6accca-09d2-462d-b7dc-777dfa52d06b","display":"Erythrocyte distribution width [Entitic volume] by Automated count"},{"reference":"Observation/0ac11784-c312-44c4-9942-b7d2beb26056","display":"Platelets [#/volume] in Blood by Automated count"},{"reference":"Observation/d29445ad-c8cf-4ced-b59a-a4683c04ce51","display":"Platelet distribution width [Entitic volume] in Blood by Automated count"},{"reference":"Observation/c80a8ad6-4bf5-476e-b3ef-09d812e14ded","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Encounter.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Encounter.ndjson new file mode 100644 index 000000000000..b0bdae6dc4c0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Encounter.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"Encounter","id":"7b569404-f475-45d9-94d3-b4e31fece687","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2010-05-19T14:15:05-07:00","end":"2010-05-19T14:45:05-07:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:45:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"ebe5ce5b-57bb-4368-93d6-0b7a0da073a1","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"3a99bba3-f304-4d33-826a-018499a08f5a","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2011-06-09T14:15:05-07:00","end":"2011-06-09T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"6245224b-6d15-448a-8e56-db606acf21d8","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"4ef1ef04-c4f7-4739-b722-e668ec9f6c35","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2014-06-12T14:15:05-07:00","end":"2014-06-12T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"7b2e1b32-79cc-412f-b8e7-97997da45089","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"37cce86a-e35b-4879-b8ed-4f22b10b2f7c","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:41:05-07:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"10509002","display":"Acute bronchitis (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"74003abb-cee0-4151-8469-4a815fff57aa","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for check up (procedure)"}],"text":"Encounter for check up (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"96baca61-8639-457b-bd48-fa32511b5985","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"187a1ecd-b5f5-4566-b39d-ae646062cfbc","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"EMER"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"50849002","display":"Emergency room admission (procedure)"}],"text":"Emergency room admission (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"522f6ab4-fd6e-42d4-b7c1-c25c99e35d3a","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185349003","display":"Encounter for 'check-up'"}],"text":"Encounter for 'check-up'"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2016-12-29T13:15:05-08:00","end":"2016-12-29T13:30:05-08:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"58150001","display":"Fracture of clavicle"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"fc81047d-ee3a-4857-b96c-0a091e957d7a","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"162673000","display":"General examination of patient (procedure)"}],"text":"General examination of patient (procedure)"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}}],"period":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:45:05-07:00"},"serviceProvider":{"reference":"Organization/58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}} +{"resourceType":"Encounter","id":"6fb5a1fb-c93c-4648-bb60-db870634bbf2","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"698314001","display":"Consultation for treatment"}],"text":"Consultation for treatment"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"89c75588-d3a0-4ff8-8202-665ff4349fa7","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"IMP"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"305408004","display":"Admission to surgical department"}],"text":"Admission to surgical department"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-10T15:15:05-08:00"},"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} +{"resourceType":"Encounter","id":"99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9","status":"finished","class":{"system":"http://terminology.hl7.org/CodeSystem/v3-ActCode","code":"AMB"},"type":[{"coding":[{"system":"http://snomed.info/sct","code":"185345009","display":"Encounter for symptom"}],"text":"Encounter for symptom"}],"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"participant":[{"individual":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"}}],"period":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:45:05-08:00"},"reason":[{"coding":[{"system":"http://snomed.info/sct","code":"43878008","display":"Streptococcal sore throat (disorder)"}]}],"serviceProvider":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ExplanationOfBenefit.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ExplanationOfBenefit.ndjson new file mode 100644 index 000000000000..7a6f130333c9 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ExplanationOfBenefit.ndjson @@ -0,0 +1,16 @@ +{"resourceType":"ExplanationOfBenefit","id":"26ef8891-a160-4c1e-8072-f974638866f6","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"462a2021-f2c0-45cc-85ec-3338664ebd9e"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2010-05-19T14:45:05-07:00","end":"2011-05-19T14:45:05-07:00"},"provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1","code":"Y","display":"Diagnosis present at time of admission"}}],"sequence":1,"diagnosisReference":{"reference":"Condition/f8fe7590-249c-4b93-8bf0-275f2feb6b81"},"type":[{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/diagnosis-type","code":"principal"}]}]}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"a0fba859-3a95-42b3-9a74-3c46be7cfcb7","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"bf23b373-969f-4bc7-89a2-b91d321f54d1"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2010-06-03T14:45:05-07:00","end":"2011-06-03T14:45:05-07:00"},"provider":{"identifier":{"value":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}},"organization":{"identifier":{"value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":3,"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"430193006"}]},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":100.96600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":403.86400000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":504.83,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":504.83,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":516.2800000000001,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"90925926-2c5e-4592-bc67-e5a951052c94","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"2bf9a87f-6c7e-4f56-b63f-4a5675faefb7"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2011-03-11T13:30:05-08:00","end":"2012-03-11T13:30:05-07:00"},"created":"2011-03-11T13:30:05-08:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/2bf9a87f-6c7e-4f56-b63f-4a5675faefb7"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2011-03-11T13:15:05-08:00","end":"2011-03-11T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/ebe5ce5b-57bb-4368-93d6-0b7a0da073a1"}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"d457c5c5-bd89-48f6-9848-e703fd3b5b42","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"0dd76242-6c52-45ce-9e13-84e635c25f4a"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2011-06-09T14:30:05-07:00","end":"2012-06-09T14:30:05-07:00"},"provider":{"identifier":{"value":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}},"organization":{"identifier":{"value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"9a3c5807-b876-45c8-aba3-565f05dec4c8","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"7f84d563-6fb1-4339-8432-766672b1737b"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2013-02-28T13:30:05-08:00","end":"2014-02-28T13:30:05-08:00"},"created":"2013-02-28T13:30:05-08:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/7f84d563-6fb1-4339-8432-766672b1737b"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2013-02-28T13:15:05-08:00","end":"2013-02-28T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/6245224b-6d15-448a-8e56-db606acf21d8"}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"66a5b348-a9c5-40f3-8096-c20ace45c3bb","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"9e3e2f01-530d-40b1-9795-eba32a97158b"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2014-06-12T14:30:05-07:00","end":"2015-06-12T14:30:05-07:00"},"provider":{"identifier":{"value":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}},"organization":{"identifier":{"value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":3,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":4,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":5,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":449.66400000000004,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"1c795fd2-353b-4594-93de-af5b191f1b2b","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"0623afa7-5753-450b-9124-c93df6a8a4f7"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-02-18T13:30:05-08:00","end":"2016-02-18T13:30:05-08:00"},"created":"2015-02-18T13:30:05-08:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/0623afa7-5753-450b-9124-c93df6a8a4f7"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2015-02-18T13:15:05-08:00","end":"2015-02-18T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/7b2e1b32-79cc-412f-b8e7-97997da45089"}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"6a4aea36-e0fc-41b3-8d2d-202653cf2aec","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"abafc9ba-edc7-4eff-98fc-e41d8785c27b"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2015-04-12T14:41:05-07:00","end":"2016-04-12T14:41:05-07:00"},"provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1","code":"Y","display":"Diagnosis present at time of admission"}}],"sequence":1,"diagnosisReference":{"reference":"Condition/5953e3df-ed6c-4509-8d56-04c745817bd0"},"type":[{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/diagnosis-type","code":"principal"}]}]}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":3,"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"399208008"}]},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":1307.116,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":5228.464,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":6535.58,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":6535.58,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":5228.464,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"88ae65ef-0f73-460c-a3d8-9a5dabb81440","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"9bd3bd28-4e88-4d8a-8510-ea791953455b"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2015-04-23T14:30:05-07:00","end":"2016-04-23T14:30:05-07:00"},"created":"2015-04-23T14:30:05-07:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/9bd3bd28-4e88-4d8a-8510-ea791953455b"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"}]},{"sequence":2,"informationLinkId":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]},"reason":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","code":"A","display":"Allowed"}]}}]},{"sequence":3,"informationLinkId":[2],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2015-04-23T14:15:05-07:00","end":"2015-04-23T14:30:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"net":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]},"reason":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","code":"A","display":"Allowed"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":224.83200000000002,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"198da8a5-4987-4df0-9be0-c74919811d87","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"36c0e892-2f4c-4387-8587-7fe004a6bafc"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-02-13T13:30:05-08:00","end":"2017-02-13T13:30:05-08:00"},"created":"2016-02-13T13:30:05-08:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/36c0e892-2f4c-4387-8587-7fe004a6bafc"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2016-02-13T13:15:05-08:00","end":"2016-02-13T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/96baca61-8639-457b-bd48-fa32511b5985"}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"5cc1e2c3-0bd6-4257-9a23-ba6629b034d0","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"d69ba5dd-6da7-4d7b-8cce-63b1ed0c5134"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2016-10-30T16:45:05-07:00","end":"2017-10-30T16:45:05-07:00"},"created":"2016-10-30T16:45:05-07:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/d69ba5dd-6da7-4d7b-8cce-63b1ed0c5134"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1","code":"Y","display":"Diagnosis present at time of admission"}}],"sequence":1,"diagnosisReference":{"reference":"Condition/0aa16b8a-bce6-4362-8312-879d7ccf29f8"},"type":[{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/diagnosis-type","code":"principal"}]}]}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"23","display":"Emergency Room"}]},"encounter":[{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"}]},{"sequence":2,"diagnosisLinkId":[1],"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"23","display":"Emergency Room"}]}},{"sequence":3,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"168594001"}]},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"23","display":"Emergency Room"}]},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":103.33,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":413.32,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]},"reason":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","code":"A","display":"Allowed"}]}}]},{"sequence":4,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"305428000"}]},"servicedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T16:45:05-07:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"23","display":"Emergency Room"}]},"net":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":103.33,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":413.32,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":516.65,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]},"reason":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","code":"A","display":"Allowed"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":826.64,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"40a1df7e-d7fb-446b-8a76-99c8b3083cdc","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"4cb0502f-3892-41f6-ac59-49d31dc28193"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2016-12-29T13:30:05-08:00","end":"2017-12-29T13:30:05-08:00"},"provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"6428ae9b-f547-4924-8a74-40eccc8838b7","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"f7183ce1-f163-40b3-8ed8-0bcff268937f"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2017-06-15T14:45:05-07:00","end":"2018-06-15T14:45:05-07:00"},"provider":{"identifier":{"value":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8"}},"organization":{"identifier":{"value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":28.104000000000003,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":112.41600000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":140.52,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":3,"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"430193006"}]},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"22","display":"Outpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":96.404,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":385.616,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":482.02,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":482.02,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":498.032,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"722adc6a-fc27-4348-8d93-bd12b951d1f0","contained":[{"resourceType":"ReferralRequest","id":"1","status":"completed","intent":"order","subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"requester":{"agent":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}},"recipient":[{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}}]},{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"e72892e7-dbe8-45ce-b6bf-4d0952c618e3"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"start":"2018-02-02T13:30:05-08:00","end":"2019-02-02T13:30:05-08:00"},"created":"2018-02-02T13:30:05-08:00","provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"referral":{"reference":"#1"},"claim":{"reference":"Claim/e72892e7-dbe8-45ce-b6bf-4d0952c618e3"},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"sequence":1,"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd","code":"1","display":"Medical care"}]},"servicedPeriod":{"start":"2018-02-02T13:15:05-08:00","end":"2018-02-02T13:30:05-08:00"},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"encounter":[{"reference":"Encounter/6fb5a1fb-c93c-4648-bb60-db870634bbf2"}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"73863e2b-5854-4742-9106-e95d3ea46813","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-ime-op-clm-val-amt-extension","valueMoney":{"value":400.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-dsh-op-clm-val-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pass-thru-per-diem-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-tot-pps-cptl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-bene-ip-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-bene-pta-coinsrnc-lblty-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-ip-ncvrd-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-ip-tot-ddctn-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-cptl-dsprprtnt-shr-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-cptl-excptn-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-cptl-fsp-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-cptl-ime-amt-extension","valueMoney":{"value":400.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-cptl-outlier-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-pps-old-cptl-hld-hrmls-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-drg-outlier-aprvd-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"8d5d5189-0c2c-4bf9-930a-7bc245908702"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2018-02-10T15:15:05-08:00","end":"2019-02-10T15:15:05-08:00"},"provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"service":{"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"287664005"}]},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":1872.8980000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":7491.592000000001,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":9364.49,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":9364.49,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":7491.592000000001,"system":"urn:iso:std:iso:4217","code":"USD"}}} +{"resourceType":"ExplanationOfBenefit","id":"e030c902-14f9-4fdc-8268-b8caac5c378f","meta":{"profile":["https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim"]},"contained":[{"resourceType":"Coverage","id":"coverage","type":{"text":"Medicaid"}}],"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd","code":"N","display":"All other reasons for non-payment"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension","valueMoney":{"value":0.0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension","valueIdentifier":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num","value":"002000"}}],"identifier":[{"system":"https://bluebutton.cms.gov/resources/variables/clm_id","value":"d220ef38-fff4-45e8-94e0-da6716f84ff6"},{"system":"https://bluebutton.cms.gov/resources/identifier/claim-group","value":"99999999999"}],"status":"active","type":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd","code":"71","display":"Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim"},{"system":"https://bluebutton.cms.gov/resources/codesystem/eob-type","code":"CARRIER","display":"EOB Type"},{"system":"http://hl7.org/fhir/ex-claimtype","code":"professional","display":"Claim Type"},{"system":"https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd","code":"O","display":"Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)"}]},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"billablePeriod":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd","code":"3","display":"Final Bill"}}],"start":"2018-12-11T13:45:05-08:00","end":"2019-12-11T13:45:05-08:00"},"provider":{"identifier":{"value":"c16820ae-2954-32d4-863c-e9ceb741154c"}},"organization":{"identifier":{"value":"465de31f-3098-365c-af70-48a071e1f5aa"}},"careTeam":[{"sequence":1,"provider":{"identifier":{"system":"http://hl7.org/fhir/sid/us-npi","value":"99999999"}},"role":{"coding":[{"system":"http://hl7.org/fhir/claimcareteamrole","code":"primary","display":"Primary Care Practitioner"}]}}],"diagnosis":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension","valueCoding":{"system":"https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1","code":"Y","display":"Diagnosis present at time of admission"}}],"sequence":1,"diagnosisReference":{"reference":"Condition/53dac70f-c5cc-4880-a96e-fdb157db8e3b"},"type":[{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/diagnosis-type","code":"principal"}]}]}],"insurance":{"coverage":{"reference":"#coverage"}},"item":[{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":1,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":2,"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]}},{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension","valueQuantity":{"value":0}}],"sequence":3,"service":{"extension":[{"url":"https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension","valueCoding":{"system":"https://www.accessdata.fda.gov/scripts/cder/ndc","code":"0624","display":"Dummy"}}],"coding":[{"system":"http://snomed.info/sct","version":"v1","code":"117015009"}]},"locationCodeableConcept":{"coding":[{"system":"https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd","code":"21","display":"Inpatient Hospital"}]},"adjudication":[{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt","display":"Line Beneficiary Coinsurance Amount"}]},"amount":{"value":391.722,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt","display":"Line Provider Payment Amount"}]},"amount":{"value":1566.888,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt","display":"Line Submitted Charge Amount"}]},"amount":{"value":1958.61,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt","display":"Line Allowed Charge Amount"}]},"amount":{"value":1958.61,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt","display":"Line Beneficiary Part B Deductible Amount"}]},"amount":{"value":0,"system":"urn:iso:std:iso:4217","code":"USD"}},{"category":{"coding":[{"system":"https://bluebutton.cms.gov/resources/codesystem/adjudication","code":"https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd","display":"Line Processing Indicator Code"}]}}]}],"totalCost":{"value":129.16,"system":"urn:iso:std:iso:4217","code":"USD"},"payment":{"amount":{"value":1566.888,"system":"urn:iso:std:iso:4217","code":"USD"}}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ImagingStudy.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ImagingStudy.ndjson new file mode 100644 index 000000000000..9e226f86482e --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/ImagingStudy.ndjson @@ -0,0 +1 @@ +{"resourceType":"ImagingStudy","id":"992ab51b-103c-4d81-8582-6b54a83342ea","uid":"urn:oid:1.2.840.99999999.22502730.1589831196459","patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"started":"2016-10-30T14:15:05-07:00","numberOfSeries":1,"numberOfInstances":1,"series":[{"uid":"urn:oid:1.2.840.99999999.1.65971777.1589831196459","number":1,"modality":{"system":"http://dicom.nema.org/resources/ontology/DCM","code":"DX","display":"Digital Radiography"},"numberOfInstances":1,"availability":"UNAVAILABLE","bodySite":{"system":"http://snomed.info/sct","code":"51299004","display":"Clavicle"},"started":"2016-10-30T14:15:05-07:00","instance":[{"uid":"urn:oid:1.2.840.99999999.1.1.48010377.1589831196459","number":1,"sopClass":"urn:oid:1.2.840.10008.5.1.4.1.1.1.1","title":"Image of clavicle"}]}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Immunization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Immunization.ndjson new file mode 100644 index 000000000000..03e078d441ce --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Immunization.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"Immunization","id":"293b214e-ae9b-4053-8ed1-becbe613de18","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"date":"2010-06-03T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"c04ae3a8-1678-404b-8fb7-2d249ba82839","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"date":"2011-06-09T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"60ab5005-b572-498c-9e32-a5acdc26e3f8","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"date":"2014-06-12T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"dc26c48b-cdd3-4384-a0f2-02d25682a02d","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"113","display":"Td (adult) preservative free"}],"text":"Td (adult) preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"date":"2014-06-12T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"52ebf7c5-8af2-4f91-9e9a-8b54de12f857","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"date":"2014-06-12T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"ded2b0fb-56ed-4245-85c8-0b2a02ff4b53","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"114","display":"meningococcal MCV4P"}],"text":"meningococcal MCV4P"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"date":"2014-06-12T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"8ea94a08-14b2-4b18-9f70-7d5f8ae729f7","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"date":"2015-04-23T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"8ca4d799-58a3-4930-a35c-c6f5a1555173","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"43","display":"Hep B, adult"}],"text":"Hep B, adult"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"date":"2015-04-23T14:15:05-07:00","primarySource":true} +{"resourceType":"Immunization","id":"69152acf-13e8-4265-bd5f-7a2df80a08f2","status":"completed","notGiven":false,"vaccineCode":{"coding":[{"system":"http://hl7.org/fhir/sid/cvx","code":"140","display":"Influenza, seasonal, injectable, preservative free"}],"text":"Influenza, seasonal, injectable, preservative free"},"patient":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"encounter":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"date":"2017-06-15T14:15:05-07:00","primarySource":true} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/MedicationRequest.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/MedicationRequest.ndjson new file mode 100644 index 000000000000..718327c2cfcc --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/MedicationRequest.ndjson @@ -0,0 +1,9 @@ +{"resourceType":"MedicationRequest","id":"df4ce22a-1ae5-429f-8d76-5a503a3921ab","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/7b569404-f475-45d9-94d3-b4e31fece687"},"authoredOn":"2010-05-19T14:15:05-07:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}},"reasonReference":[{"reference":"Condition/f8fe7590-249c-4b93-8bf0-275f2feb6b81"}]} +{"resourceType":"MedicationRequest","id":"8c7b34ec-760b-412b-bfd4-ec560b2ac998","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"751905","display":"Trinessa 28 Day Pack"}],"text":"Trinessa 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/ebe5ce5b-57bb-4368-93d6-0b7a0da073a1"},"authoredOn":"2011-03-11T13:15:05-08:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}}} +{"resourceType":"MedicationRequest","id":"546161c7-df48-41ae-a25f-2dbbf39729be","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"748856","display":"Yaz 28 Day Pack"}],"text":"Yaz 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/6245224b-6d15-448a-8e56-db606acf21d8"},"authoredOn":"2013-02-28T13:15:05-08:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}}} +{"resourceType":"MedicationRequest","id":"c9970c0c-236e-4431-bfe9-c08cdd622411","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"1534809","display":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"}],"text":"168 HR Ethinyl Estradiol 0.00146 MG/HR / norelgestromin 0.00625 MG/HR Transdermal System"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/7b2e1b32-79cc-412f-b8e7-97997da45089"},"authoredOn":"2015-02-18T13:15:05-08:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}}} +{"resourceType":"MedicationRequest","id":"6cb4c447-aa9d-4bfc-89f3-e424dab1fd2a","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"313782","display":"Acetaminophen 325 MG Oral Tablet"}],"text":"Acetaminophen 325 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"},"authoredOn":"2015-04-12T14:15:05-07:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}},"reasonReference":[{"reference":"Condition/5953e3df-ed6c-4509-8d56-04c745817bd0"}]} +{"resourceType":"MedicationRequest","id":"e42328cf-bb27-470f-9cff-a9dcb8fd7485","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"978950","display":"Natazia 28 Day Pack"}],"text":"Natazia 28 Day Pack"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/96baca61-8639-457b-bd48-fa32511b5985"},"authoredOn":"2016-02-13T13:15:05-08:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}}} +{"resourceType":"MedicationRequest","id":"b8c23eb0-d9b6-4413-8293-62c13b47e2ab","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"861467","display":"Meperidine Hydrochloride 50 MG Oral Tablet"}],"text":"Meperidine Hydrochloride 50 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"authoredOn":"2016-10-30T14:15:05-07:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}},"dosageInstruction":[{"sequence":1,"timing":{"repeat":{"frequency":1,"period":4.0,"periodUnit":"h"}},"asNeededBoolean":false,"doseQuantity":{"value":1.0}}]} +{"resourceType":"MedicationRequest","id":"22ba50dd-b723-4771-9247-2b25e566da68","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"310965","display":"Ibuprofen 200 MG Oral Tablet"}],"text":"Ibuprofen 200 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"authoredOn":"2016-10-30T14:15:05-07:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}},"dosageInstruction":[{"sequence":1,"asNeededBoolean":true}]} +{"resourceType":"MedicationRequest","id":"f313632b-f3da-4507-a3de-954c156bd799","status":"stopped","intent":"order","medicationCodeableConcept":{"coding":[{"system":"http://www.nlm.nih.gov/research/umls/rxnorm","code":"834102","display":"Penicillin V Potassium 500 MG Oral Tablet"}],"text":"Penicillin V Potassium 500 MG Oral Tablet"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"},"authoredOn":"2018-12-11T13:15:05-08:00","requester":{"agent":{"reference":"Practitioner/c16820ae-2954-32d4-863c-e9ceb741154c"},"onBehalfOf":{"reference":"Organization/465de31f-3098-365c-af70-48a071e1f5aa"}},"reasonReference":[{"reference":"Condition/f8fe7590-249c-4b93-8bf0-275f2feb6b81"},{"reference":"Condition/53dac70f-c5cc-4880-a96e-fdb157db8e3b"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Observation.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Observation.ndjson new file mode 100644 index 000000000000..56e2b92c6649 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Observation.ndjson @@ -0,0 +1,54 @@ +{"resourceType":"Observation","id":"1bd4f633-9348-41ae-aaa9-7870c0e2c840","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":153.6,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"a42f60ca-c6c0-48e2-963a-17d022452b98","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":1,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"df7d1c32-d860-45a3-bda4-32f32d866a32","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":55.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"29c6ee5d-8755-449d-b9a2-dd15c74d225b","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":23.41,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"839d525b-6730-42e2-b87f-1a4da2afab5f","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":71.788,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"c2505847-c2b3-4cdd-9042-527fa5413427","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":72,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":120,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"71a18dd9-86bb-4592-98a0-c5f1a1a8432a","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"7b451550-d8e1-4876-a7c4-dcb2b239ed77","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"3e63fc53-2b1d-4aa8-a258-4c6f0b913426","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"effectiveDateTime":"2010-06-03T14:15:05-07:00","issued":"2010-06-03T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"34434c43-6a31-4338-812f-2f8b9f3fe6d1","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":153.7,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"694f4732-4f02-4885-a62d-49a569cca6a7","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"894cddeb-1759-4714-a7db-16c2ecad0be7","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":62.1,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"0e9e67fc-c729-479e-9284-56399ccfb04b","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":26.3,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"46d8334e-7631-4a4e-936b-b451fdf795f9","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"59576-9","display":"Body mass index (BMI) [Percentile] Per age and gender"}],"text":"Body mass index (BMI) [Percentile] Per age and gender"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":85.491,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"2340483d-6118-4dfd-9256-37b815fc623d","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":88,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":121,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"21ccc32a-5d37-4dc7-b1bf-aefd6c930708","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":93,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"d6c19b7f-3e0b-4841-8ce1-3cbd7f7459f8","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueQuantity":{"value":13,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"c8ba5ba3-1bcd-4043-aade-0a9b109724b5","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/3a99bba3-f304-4d33-826a-018499a08f5a"},"effectiveDateTime":"2011-06-09T14:15:05-07:00","issued":"2011-06-09T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"bde88c9f-40fa-44d0-a465-05523c0f25c6","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"f3ec4455-433a-468a-984f-ddd1f1edef66","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"4e4efc47-10c1-43d7-9481-a7424a2b06e0","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":69.2,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"77fd7fe3-ae38-4ec4-a18d-a7e24ed97fc7","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":29.25,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"76bac1a4-fe2a-472c-af3e-f5c1234eef5a","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":87,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":122,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"af1fa8f5-efef-4b15-959d-6a66f9a5bf16","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":90,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"ed5a9f51-3e45-44d1-8208-604cac028376","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":16,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"a71330cb-bdf0-4383-9411-0e3372da9577","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"6690-2","display":"Leukocytes [#/volume] in Blood by Automated count"}],"text":"Leukocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":7.2857,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"f74b75ea-1ad6-4418-ae8b-2bd62e875e21","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"789-8","display":"Erythrocytes [#/volume] in Blood by Automated count"}],"text":"Erythrocytes [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":4.7109,"unit":"10*6/uL","system":"http://unitsofmeasure.org","code":"10*6/uL"}} +{"resourceType":"Observation","id":"573fb355-b0bf-4af5-98d8-ead5389eb3cc","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"718-7","display":"Hemoglobin [Mass/volume] in Blood"}],"text":"Hemoglobin [Mass/volume] in Blood"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":15.964,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"d750ec32-b61a-40da-b58d-fff11de5bf27","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"4544-3","display":"Hematocrit [Volume Fraction] of Blood by Automated count"}],"text":"Hematocrit [Volume Fraction] of Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":45.934,"unit":"%","system":"http://unitsofmeasure.org","code":"%"}} +{"resourceType":"Observation","id":"ab86a392-7cca-4625-97d4-421aeaa14382","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"787-2","display":"MCV [Entitic volume] by Automated count"}],"text":"MCV [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":94.793,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"0bb6c8de-bf30-43bd-9ae7-b56bc3c74914","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"785-6","display":"MCH [Entitic mass] by Automated count"}],"text":"MCH [Entitic mass] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":28.937,"unit":"pg","system":"http://unitsofmeasure.org","code":"pg"}} +{"resourceType":"Observation","id":"ff40e33c-caee-4181-b71f-cf9a12f7a24f","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"786-4","display":"MCHC [Mass/volume] by Automated count"}],"text":"MCHC [Mass/volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":33.479,"unit":"g/dL","system":"http://unitsofmeasure.org","code":"g/dL"}} +{"resourceType":"Observation","id":"2e6accca-09d2-462d-b7dc-777dfa52d06b","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"21000-5","display":"Erythrocyte distribution width [Entitic volume] by Automated count"}],"text":"Erythrocyte distribution width [Entitic volume] by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":43.303,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"0ac11784-c312-44c4-9942-b7d2beb26056","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"777-3","display":"Platelets [#/volume] in Blood by Automated count"}],"text":"Platelets [#/volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":173.89,"unit":"10*3/uL","system":"http://unitsofmeasure.org","code":"10*3/uL"}} +{"resourceType":"Observation","id":"d29445ad-c8cf-4ced-b59a-a4683c04ce51","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"32207-3","display":"Platelet distribution width [Entitic volume] in Blood by Automated count"}],"text":"Platelet distribution width [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":413.95,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"c80a8ad6-4bf5-476e-b3ef-09d812e14ded","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"laboratory","display":"laboratory"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"32623-1","display":"Platelet mean volume [Entitic volume] in Blood by Automated count"}],"text":"Platelet mean volume [Entitic volume] in Blood by Automated count"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueQuantity":{"value":10.07,"unit":"fL","system":"http://unitsofmeasure.org","code":"fL"}} +{"resourceType":"Observation","id":"99f42a3c-5eb5-49db-8448-b24cfc67b24d","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/4ef1ef04-c4f7-4739-b722-e668ec9f6c35"},"effectiveDateTime":"2014-06-12T14:15:05-07:00","issued":"2014-06-12T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"7c5bd16f-53b7-4d34-8c81-c0ec8b52bbe1","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"5e71e502-49b6-4aa4-8fb1-a1b8b9a9b16c","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":2,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"cdd7d477-1300-4b05-b401-f496a6a5b6fd","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":70.3,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"b3b07e6c-6406-49e2-b36d-8df17c3beed0","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":29.73,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"75ffe915-fb72-49ad-a8ca-3a775a2c1bdd","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":79,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":129,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"6068a754-23b6-4b25-b2cb-a445ef6a00ad","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":90,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"036fb690-7a46-43d0-9b10-bea7e5471816","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueQuantity":{"value":15,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"e9e4c78f-5076-451e-a1f9-a3bf65ceea48","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/74003abb-cee0-4151-8469-4a815fff57aa"},"effectiveDateTime":"2015-04-23T14:15:05-07:00","issued":"2015-04-23T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"75c27e1b-e4b4-4273-bf81-ac672aec77ff","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8302-2","display":"Body Height"}],"text":"Body Height"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":153.8,"unit":"cm","system":"http://unitsofmeasure.org","code":"cm"}} +{"resourceType":"Observation","id":"656b0220-e028-48b0-8ce2-3b9e63c0940a","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72514-3","display":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"}],"text":"Pain severity - 0-10 verbal numeric rating [Score] - Reported"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":0,"unit":"{score}","system":"http://unitsofmeasure.org","code":"{score}"}} +{"resourceType":"Observation","id":"dda3a68b-c604-4ac6-b033-aef7f3dfc5a4","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"29463-7","display":"Body Weight"}],"text":"Body Weight"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66.8,"unit":"kg","system":"http://unitsofmeasure.org","code":"kg"}} +{"resourceType":"Observation","id":"9eed66ed-362b-48bd-b464-b089990ad19a","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"39156-5","display":"Body Mass Index"}],"text":"Body Mass Index"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":28.22,"unit":"kg/m2","system":"http://unitsofmeasure.org","code":"kg/m2"}} +{"resourceType":"Observation","id":"26c30c4b-0461-4b6f-a84f-e6b075438799","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"85354-9","display":"Blood Pressure"}],"text":"Blood Pressure"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","component":[{"code":{"coding":[{"system":"http://loinc.org","code":"8462-4","display":"Diastolic Blood Pressure"}],"text":"Diastolic Blood Pressure"},"valueQuantity":{"value":69,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}},{"code":{"coding":[{"system":"http://loinc.org","code":"8480-6","display":"Systolic Blood Pressure"}],"text":"Systolic Blood Pressure"},"valueQuantity":{"value":111,"unit":"mm[Hg]","system":"http://unitsofmeasure.org","code":"mm[Hg]"}}]} +{"resourceType":"Observation","id":"3581b2d3-4c7a-46f0-81c5-33f249738522","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8867-4","display":"Heart rate"}],"text":"Heart rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":66,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"1258d6ee-aed7-48ac-b8a5-058940f9d48b","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"9279-1","display":"Respiratory rate"}],"text":"Respiratory rate"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueQuantity":{"value":12,"unit":"/min","system":"http://unitsofmeasure.org","code":"/min"}} +{"resourceType":"Observation","id":"7339a567-e371-4a13-8fb9-b6c2a61938ac","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"survey","display":"survey"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"72166-2","display":"Tobacco smoking status NHIS"}],"text":"Tobacco smoking status NHIS"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"effectiveDateTime":"2017-06-15T14:15:05-07:00","issued":"2017-06-15T14:15:05.225-07:00","valueCodeableConcept":{"coding":[{"system":"http://snomed.info/sct","code":"266919005","display":"Never smoker"}],"text":"Never smoker"}} +{"resourceType":"Observation","id":"1902e0c7-6e2b-42b0-9f37-65e94511286d","status":"final","category":[{"coding":[{"system":"http://hl7.org/fhir/observation-category","code":"vital-signs","display":"vital-signs"}]}],"code":{"coding":[{"system":"http://loinc.org","code":"8310-5","display":"Body temperature"}],"text":"Body temperature"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"},"effectiveDateTime":"2018-12-11T13:15:05-08:00","issued":"2018-12-11T13:15:05.225-08:00","valueQuantity":{"value":39.024,"unit":"Cel","system":"http://unitsofmeasure.org","code":"Cel"}} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Organization.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Organization.ndjson new file mode 100644 index 000000000000..06c5160f19af --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Organization.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Organization","id":"465de31f-3098-365c-af70-48a071e1f5aa","identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"465de31f-3098-365c-af70-48a071e1f5aa"}],"type":[{"coding":[{"system":"http://hl7.org/fhir/organization-type","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"}],"name":"METROWEST MEDICAL CENTER","telecom":[{"system":"phone","value":"5083831000"}],"address":[{"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/geolocation","extension":[{"url":"latitude","valueDecimal":42.307905},{"url":"longitude","valueDecimal":-71.436196}]}],"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}]} +{"resourceType":"Organization","id":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8","identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"58fe1815-1e8a-38ed-a91a-17d4ef18c8d8"}],"type":[{"coding":[{"system":"http://hl7.org/fhir/organization-type","code":"prov","display":"Healthcare Provider"}],"text":"Healthcare Provider"}],"name":"PCP68975","telecom":[{"system":"phone","value":"508-881-4368"}],"address":[{"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/geolocation","extension":[{"url":"latitude","valueDecimal":42.257754999999996},{"url":"longitude","valueDecimal":-71.473526}]}],"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Patient.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Patient.ndjson new file mode 100644 index 000000000000..3243439fb2f8 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Patient.ndjson @@ -0,0 +1 @@ +{"resourceType":"Patient","id":"1416dec1-f4b1-4b48-b7f4-650e8f67499c","text":{"status":"generated","div":"
    Generated by Synthea.Version identifier: v2.5.0-265-gbd5a00e8\n . Person seed: 6732543839779682504 Population seed: 1589831189867
    "},"extension":[{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-race","extension":[{"url":"ombCategory","valueCoding":{"system":"urn:oid:2.16.840.1.113883.6.238","code":"2106-3","display":"White"}},{"url":"text","valueString":"White"}]},{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity","extension":[{"url":"ombCategory","valueCoding":{"system":"urn:oid:2.16.840.1.113883.6.238","code":"2186-5","display":"Not Hispanic or Latino"}},{"url":"text","valueString":"Not Hispanic or Latino"}]},{"url":"http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName","valueString":"Leanna255 Predovic534"},{"url":"http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex","valueCode":"F"},{"url":"http://hl7.org/fhir/StructureDefinition/birthPlace","valueAddress":{"city":"Southbridge","state":"Massachusetts","country":"US"}},{"url":"http://synthetichealth.github.io/synthea/disability-adjusted-life-years","valueDecimal":0.0},{"url":"http://synthetichealth.github.io/synthea/quality-adjusted-life-years","valueDecimal":27.0}],"identifier":[{"system":"https://github.com/synthetichealth/synthea","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://hl7.org/fhir/v2/0203","code":"MR","display":"Medical Record Number"}],"text":"Medical Record Number"},"system":"http://hospital.smarthealthit.org","value":"1416dec1-f4b1-4b48-b7f4-650e8f67499c"},{"type":{"coding":[{"system":"http://hl7.org/fhir/identifier-type","code":"SB","display":"Social Security Number"}],"text":"Social Security Number"},"system":"http://hl7.org/fhir/sid/us-ssn","value":"999-54-3579"},{"type":{"coding":[{"system":"http://hl7.org/fhir/v2/0203","code":"DL","display":"Driver's License"}],"text":"Driver's License"},"system":"urn:oid:2.16.840.1.113883.4.3.25","value":"S99972984"},{"type":{"coding":[{"system":"http://hl7.org/fhir/v2/0203","code":"PPN","display":"Passport Number"}],"text":"Passport Number"},"system":"http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber","value":"X39621585X"}],"name":[{"use":"official","family":"Nolan344","given":["Lorita217"],"prefix":["Ms."]}],"telecom":[{"system":"phone","value":"555-817-6998","use":"home"}],"gender":"female","birthDate":"1992-04-09","address":[{"extension":[{"url":"http://hl7.org/fhir/StructureDefinition/geolocation","extension":[{"url":"latitude","valueDecimal":42.27693107900605},{"url":"longitude","valueDecimal":-71.45741653702677}]}],"line":["330 Sawayn Parade"],"city":"Framingham","state":"Massachusetts","country":"US"}],"maritalStatus":{"coding":[{"system":"http://hl7.org/fhir/v3/MaritalStatus","code":"S","display":"S"}],"text":"S"},"multipleBirthBoolean":false,"communication":[{"language":{"coding":[{"system":"urn:ietf:bcp:47","code":"en-US","display":"English"}],"text":"English"}}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Practitioner.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Practitioner.ndjson new file mode 100644 index 000000000000..eba7b8c26455 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Practitioner.ndjson @@ -0,0 +1,2 @@ +{"resourceType":"Practitioner","id":"c16820ae-2954-32d4-863c-e9ceb741154c","identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"530"}],"active":true,"name":[{"family":"Murphy561","given":["Mari763"],"prefix":["Dr."]}],"address":[{"line":["115 LINCOLN STREET"],"city":"FRAMINGHAM","state":"MA","postalCode":"01701","country":"US"}],"gender":"female"} +{"resourceType":"Practitioner","id":"a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8","identifier":[{"system":"http://hl7.org/fhir/sid/us-npi","value":"35750"}],"active":true,"name":[{"family":"Hilpert278","given":["Cathryn51"],"prefix":["Dr."]}],"address":[{"line":["259 MAIN ST"],"city":"ASHLAND","state":"MA","postalCode":"01721-2115","country":"US"}],"gender":"female"} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Procedure.ndjson b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Procedure.ndjson new file mode 100644 index 000000000000..3a484c87be00 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/Procedure.ndjson @@ -0,0 +1,7 @@ +{"resourceType":"Procedure","id":"63702bbb-e289-4a80-9f1a-3ef5a7d15019","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/d122a0a0-c565-4e44-a83f-6b9ddd7f8cd7"},"performedPeriod":{"start":"2010-06-03T14:15:05-07:00","end":"2010-06-03T14:30:05-07:00"}} +{"resourceType":"Procedure","id":"bbedb163-5b9d-4865-bd9f-d59f7a9c21ad","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"399208008","display":"Plain chest X-ray (procedure)"}],"text":"Plain chest X-ray (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/37cce86a-e35b-4879-b8ed-4f22b10b2f7c"},"performedPeriod":{"start":"2015-04-12T14:15:05-07:00","end":"2015-04-12T14:26:05-07:00"},"reasonReference":[{"reference":"Condition/5953e3df-ed6c-4509-8d56-04c745817bd0","display":"Acute bronchitis (disorder)"}]} +{"resourceType":"Procedure","id":"565fedff-5363-4938-9b49-1a2b5b004e15","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"168594001","display":"Clavicle X-ray"}],"text":"Clavicle X-ray"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T14:45:05-07:00"}} +{"resourceType":"Procedure","id":"faae5e23-a03a-450a-ad12-50ca56eaa0e1","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"305428000","display":"Admission to orthopedic department"}],"text":"Admission to orthopedic department"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/187a1ecd-b5f5-4566-b39d-ae646062cfbc"},"performedPeriod":{"start":"2016-10-30T14:15:05-07:00","end":"2016-10-30T15:15:05-07:00"},"reasonReference":[{"reference":"Condition/0aa16b8a-bce6-4362-8312-879d7ccf29f8","display":"Fracture of clavicle"}]} +{"resourceType":"Procedure","id":"2b4a074e-e62c-4208-9ae5-be371b9208d3","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"430193006","display":"Medication Reconciliation (procedure)"}],"text":"Medication Reconciliation (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/fc81047d-ee3a-4857-b96c-0a091e957d7a"},"performedPeriod":{"start":"2017-06-15T14:15:05-07:00","end":"2017-06-15T14:30:05-07:00"}} +{"resourceType":"Procedure","id":"28e04a1a-c56a-4aca-a144-ec986f09f1aa","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"287664005","display":"Bilateral tubal ligation"}],"text":"Bilateral tubal ligation"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/89c75588-d3a0-4ff8-8202-665ff4349fa7"},"performedPeriod":{"start":"2018-02-09T13:15:05-08:00","end":"2018-02-09T15:15:05-08:00"}} +{"resourceType":"Procedure","id":"17543a9f-3947-4a3c-af28-21391ee2674a","status":"completed","code":{"coding":[{"system":"http://snomed.info/sct","code":"117015009","display":"Throat culture (procedure)"}],"text":"Throat culture (procedure)"},"subject":{"reference":"Patient/1416dec1-f4b1-4b48-b7f4-650e8f67499c"},"context":{"reference":"Encounter/99ff4b9b-41f5-41ce-ac7c-b96808cc7ed9"},"performedPeriod":{"start":"2018-12-11T13:15:05-08:00","end":"2018-12-11T13:30:05-08:00"},"reasonReference":[{"reference":"Condition/f8fe7590-249c-4b93-8bf0-275f2feb6b81","display":"Streptococcal sore throat (disorder)"},{"reference":"Condition/53dac70f-c5cc-4880-a96e-fdb157db8e3b","display":"Streptococcal sore throat (disorder)"}]} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json new file mode 100644 index 000000000000..dfe338def8e6 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json @@ -0,0 +1,234 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", + "resource": { + "resourceType": "Organization", + "id": "465de31f-3098-365c-af70-48a071e1f5aa", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "465de31f-3098-365c-af70-48a071e1f5aa" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "METROWEST MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5083831000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.307905 + }, + { + "url": "longitude", + "valueDecimal": -71.436196 + } + ] + } + ], + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "resource": { + "resourceType": "Organization", + "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 23 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 5 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 17 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP68975", + "telecom": [ + { + "system": "phone", + "value": "508-881-4368" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.257754999999996 + }, + { + "url": "longitude", + "valueDecimal": -71.473526 + } + ] + } + ], + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ] + } + }, + { + "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "resource": { + "resourceType": "Organization", + "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "WALTHAM URGENT CARE", + "telecom": [ + { + "system": "phone", + "value": "617-243-5591" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.3700513 + }, + { + "url": "longitude", + "valueDecimal": -71.2490152 + } + ] + } + ], + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ] + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json new file mode 100644 index 000000000000..9e963d958d03 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json @@ -0,0 +1,135 @@ +{ + "resourceType": "Bundle", + "type": "collection", + "entry": [ + { + "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", + "resource": { + "resourceType": "Practitioner", + "id": "c16820ae-2954-32d4-863c-e9ceb741154c", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 27 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "530" + } + ], + "active": true, + "name": [ + { + "family": "Murphy561", + "given": [ + "Mari763" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "115 LINCOLN STREET" + ], + "city": "FRAMINGHAM", + "state": "MA", + "postalCode": "01701", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "resource": { + "resourceType": "Practitioner", + "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 30 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "35750" + } + ], + "active": true, + "name": [ + { + "family": "Hilpert278", + "given": [ + "Cathryn51" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "259 MAIN ST" + ], + "city": "ASHLAND", + "state": "MA", + "postalCode": "01721-2115", + "country": "US" + } + ], + "gender": "female" + } + }, + { + "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", + "resource": { + "resourceType": "Practitioner", + "id": "861aa935-d89f-3b41-861b-b791fa112253", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93440" + } + ], + "active": true, + "name": [ + { + "family": "Reynolds644", + "given": [ + "Salena230" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "9 HOPE AVENUE" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2453", + "country": "US" + } + ], + "gender": "female" + } + } + ] +} From 35db9f7c4ed9baf7f263bd93f75bbb549d3f09ce Mon Sep 17 00:00:00 2001 From: Esun Kim Date: Tue, 19 May 2020 13:26:19 -0700 Subject: [PATCH 018/151] Add grpc-alts and grpc-netty-shaded explicitly --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 ++ sdks/java/io/google-cloud-platform/build.gradle | 2 ++ 2 files changed, 4 insertions(+) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index ac53a23daf4a..0adb35c5ba20 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -481,6 +481,7 @@ class BeamModulePlugin implements Plugin { google_oauth_client : "com.google.oauth-client:google-oauth-client:$google_oauth_clients_version", google_oauth_client_java6 : "com.google.oauth-client:google-oauth-client-java6:$google_oauth_clients_version", grpc_all : "io.grpc:grpc-all:$grpc_version", + grpc_alts : "io.grpc:grpc-alts:$grpc_version", grpc_auth : "io.grpc:grpc-auth:$grpc_version", grpc_core : "io.grpc:grpc-core:$grpc_version", grpc_context : "io.grpc:grpc-context:$grpc_version", @@ -489,6 +490,7 @@ class BeamModulePlugin implements Plugin { grpc_protobuf : "io.grpc:grpc-protobuf:$grpc_version", grpc_protobuf_lite : "io.grpc:grpc-protobuf-lite:$grpc_version", grpc_netty : "io.grpc:grpc-netty:$grpc_version", + grpc_netty_shaded : "io.grpc:grpc-netty-shaded:$grpc_version", grpc_stub : "io.grpc:grpc-stub:$grpc_version", guava : "com.google.guava:guava:$guava_version", guava_testlib : "com.google.guava:guava-testlib:$guava_version", diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index c6c8e93927a2..e36bac6fa402 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -56,11 +56,13 @@ dependencies { compile library.java.google_http_client compile library.java.google_http_client_jackson2 compile library.java.grpc_all + compile library.java.grpc_alts compile library.java.grpc_auth compile library.java.grpc_core compile library.java.grpc_context compile library.java.grpc_grpclb compile library.java.grpc_netty + compile library.java.grpc_netty_shaded compile library.java.grpc_stub compile library.java.grpc_google_cloud_pubsub_v1 compile library.java.guava From 3b50369b089a94eb961dcc170bee7b97c0345805 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Tue, 19 May 2020 17:15:22 -0700 Subject: [PATCH 019/151] [BEAM-10038] Add script to mass-comment Jenkins triggers on PR --- release/src/main/scripts/mass_comment.py | 141 +++++++++++++++++++++++ 1 file changed, 141 insertions(+) create mode 100644 release/src/main/scripts/mass_comment.py diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py new file mode 100644 index 000000000000..bd9e26aa3789 --- /dev/null +++ b/release/src/main/scripts/mass_comment.py @@ -0,0 +1,141 @@ +# +# 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. +# + +"""Script for mass-commenting Jenkins test triggers on a Beam PR.""" + +import itertools +import os +import socket +import sys +import time +import traceback +import re +import requests +from datetime import datetime + + +COMMENTS_TO_ADD=[ + "Run Go PostCommit", + "Run Java PostCommit", + "Run Java PortabilityApi PostCommit", + "Run Java Flink PortableValidatesRunner Batch", + "Run Java Flink PortableValidatesRunner Streaming", + "Run Apex ValidatesRunner", + "Run Dataflow ValidatesRunner", + "Run Flink ValidatesRunner", + "Run Gearpump ValidatesRunner", + "Run Dataflow PortabilityApi ValidatesRunner", + "Run Samza ValidatesRunner", + "Run Spark ValidatesRunner", + "Run Python Dataflow ValidatesContainer", + "Run Python Dataflow ValidatesRunner", + "Run Python 3.5 Flink ValidatesRunner", + "Run Python 2 PostCommit", + "Run Python 3.5 PostCommit", + "Run SQL PostCommit", + "Run Go PreCommit", + "Run Java PreCommit", + "Run Java_Examples_Dataflow PreCommit", + "Run JavaPortabilityApi PreCommit", + "Run Portable_Python PreCommit", + "Run PythonLint PreCommit", + "Run Python PreCommit", + "Run Python DockerBuild PreCommit" +] + +def executeGHGraphqlQuery(accessToken, query): + '''Runs graphql query on GitHub.''' + url = 'https://api.github.com/graphql' + headers = {'Authorization': 'Bearer %s' % accessToken} + r = requests.post(url=url, json={'query': query}, headers=headers) + return r.json() + +def getSubjectId(accessToken, prNumber): + query = ''' +query FindPullRequestID { + repository(owner:"apache", name:"beam") { + pullRequest(number:%s) { + id + } + } +} +''' % prNumber + response = executeGHGraphqlQuery(accessToken, query) + return response['data']['repository']['pullRequest']['id'] + +def fetchGHData(accessToken, subjectId, commentBody): + '''Fetches GitHub data required for reporting Beam metrics''' + query = ''' +mutation AddPullRequestComment { + addComment(input:{subjectId:"%s",body: "%s"}) { + commentEdge { + node { + createdAt + body + } + } + subject { + id + } + } +} +''' % (subjectId, commentBody) + return executeGHGraphqlQuery(accessToken, query) + +def postComments(accessToken, subjectId): + ''' + Main workhorse method. Fetches data from GitHub and puts it in metrics table. + ''' + + for commentBody in COMMENTS_TO_ADD: + jsonData = fetchGHData(accessToken, subjectId, commentBody) + print(jsonData) + +def probeGitHubIsUp(): + ''' + Returns True if GitHub responds to simple queries. Else returns False. + ''' + sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + result = sock.connect_ex(('github.com', 443)) + return True if result == 0 else False + + +################################################################################ +if __name__ == '__main__': + ''' + This script is supposed to be invoked directly. + However for testing purposes and to allow importing, + wrap work code in module check. + ''' + print("Started.") + + + if not probeGitHubIsUp(): + print("GitHub is unavailable, skipping fetching data.") + exit(); + + print("GitHub is available start fetching data.") + + accessToken = input("Enter your Github access token: ") + + pr = input("Enter the Beam PR number to test (e.g. 11403): ") + subjectId = getSubjectId(accessToken, pr) + + postComments(accessToken, subjectId) + print("Fetched data.") + + print('Done.') From 5dda6a86f5a7bd9881d059e28bf6bfe367486822 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Tue, 19 May 2020 20:26:28 -0400 Subject: [PATCH 020/151] Add mass comment script to release guide. --- website/www/site/content/en/contribute/release-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 95c95c2272bb..55ec1d126d9a 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -398,7 +398,7 @@ There are 2 ways to perform this verification, either running automation script( Jenkins job `beam_Release_Gradle_Build` basically run `./gradlew build -PisRelease`. This only verifies that everything builds with unit tests passing. -You can refer to [this script](https://gist.github.com/Ardagan/13e6031e8d1c9ebbd3029bf365c1a517) to mass-comment on PR. +You can use [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to mass-comment on PR. #### Verify the build succeeds From c2d6c36e03dffcbfd2387ac19862a4d9cb401494 Mon Sep 17 00:00:00 2001 From: Yichi Zhang Date: Tue, 19 May 2020 17:53:16 -0700 Subject: [PATCH 021/151] [BEAM-9603] Add timer family support to FnApiDoFnRunner --- .../sdk/transforms/reflect/DoFnInvoker.java | 8 +- .../beam/fn/harness/FnApiDoFnRunner.java | 122 ++++++++++++++---- 2 files changed, 104 insertions(+), 26 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index f8d4a365f8b1..40d2ca9c6332 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -241,6 +241,10 @@ interface ArgumentProvider { */ TimerMap timerFamily(String tagId); + /** + * Returns the timer id for the current timer of a {@link + * org.apache.beam.sdk.transforms.DoFn.TimerFamily}. + */ String timerId(DoFn doFn); } @@ -532,8 +536,8 @@ public Timer timer(String timerId) { } @Override - public TimerMap timerFamily(String tagId) { - return delegate.timerFamily(tagId); + public TimerMap timerFamily(String timerFamilyId) { + return delegate.timerFamily(timerFamilyId); } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 98b3e66ad4f6..7ee066956de6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -93,6 +93,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.StateDeclaration; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerDeclaration; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerFamilyDeclaration; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.HasProgress; @@ -460,14 +461,22 @@ public void accept(WindowedValue input) throws Exception { // Extract out relevant TimerFamilySpec information in preparation for execution. for (Map.Entry entry : parDoPayload.getTimerFamilySpecsMap().entrySet()) { - String timerFamilyId = entry.getKey(); - TimeDomain timeDomain = - DoFnSignatures.getTimerSpecOrThrow( - doFnSignature.timerDeclarations().get(timerFamilyId), doFn) - .getTimeDomain(); + String timerIdOrTimerFamilyId = entry.getKey(); + TimeDomain timeDomain; + if (timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX)) { + timeDomain = + DoFnSignatures.getTimerFamilySpecOrThrow( + doFnSignature.timerFamilyDeclarations().get(timerIdOrTimerFamilyId), doFn) + .getTimeDomain(); + } else { + timeDomain = + DoFnSignatures.getTimerSpecOrThrow( + doFnSignature.timerDeclarations().get(timerIdOrTimerFamilyId), doFn) + .getTimeDomain(); + } Coder> timerCoder = (Coder) rehydratedComponents.getCoder(entry.getValue().getTimerFamilyCoderId()); - timerFamilyInfosBuilder.put(timerFamilyId, KV.of(timeDomain, timerCoder)); + timerFamilyInfosBuilder.put(timerIdOrTimerFamilyId, KV.of(timeDomain, timerCoder)); } timerFamilyInfos = timerFamilyInfosBuilder.build(); @@ -962,16 +971,25 @@ private HandlesSplits.SplitResult trySplitForElementAndRestriction( .build()); } - private void processTimer(String timerId, TimeDomain timeDomain, Timer timer) { + private void processTimer( + String timerIdOrTimerFamilyId, TimeDomain timeDomain, Timer timer) { currentTimer = timer; currentTimeDomain = timeDomain; onTimerContext = new OnTimerContext<>(timer.getUserKey()); + String timerId = + timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX) + ? "" + : timerIdOrTimerFamilyId; + String timerFamilyId = + timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX) + ? timerIdOrTimerFamilyId + : ""; try { Iterator windowIterator = (Iterator) timer.getWindows().iterator(); while (windowIterator.hasNext()) { currentWindow = windowIterator.next(); - doFnInvoker.invokeOnTimer(timerId, "", onTimerContext); + doFnInvoker.invokeOnTimer(timerId, timerFamilyId, onTimerContext); } } finally { currentTimer = null; @@ -1037,16 +1055,15 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { BoundedWindow boundedWindow, Instant elementTimestampOrTimerHoldTimestamp, Instant elementTimestampOrTimerFireTimestamp, - PaneInfo paneInfo) { + PaneInfo paneInfo, + TimeDomain timeDomain) { this.timerId = timerId; this.userKey = userKey; this.dynamicTimerTag = dynamicTimerTag; this.elementTimestampOrTimerHoldTimestamp = elementTimestampOrTimerHoldTimestamp; this.boundedWindow = boundedWindow; this.paneInfo = paneInfo; - - TimerDeclaration timerDeclaration = doFnSignature.timerDeclarations().get(timerId); - this.timeDomain = DoFnSignatures.getTimerSpecOrThrow(timerDeclaration, doFn).getTimeDomain(); + this.timeDomain = timeDomain; switch (timeDomain) { case EVENT_TIME: @@ -1207,15 +1224,51 @@ private void output(Instant scheduledTime) { } } - private static class FnApiTimerMap implements TimerMap { - FnApiTimerMap() {} + private class FnApiTimerMap implements TimerMap { + private final String timerFamilyId; + private final K userKey; + private final TimeDomain timeDomain; + private final Instant elementTimestampOrTimerHoldTimestamp; + private final Instant elementTimestampOrTimerFireTimestamp; + private final BoundedWindow boundedWindow; + private final PaneInfo paneInfo; + + FnApiTimerMap( + String timerFamilyId, + K userKey, + BoundedWindow boundedWindow, + Instant elementTimestampOrTimerHoldTimestamp, + Instant elementTimestampOrTimerFireTimestamp, + PaneInfo paneInfo) { + this.timerFamilyId = timerFamilyId; + this.userKey = userKey; + this.elementTimestampOrTimerHoldTimestamp = elementTimestampOrTimerHoldTimestamp; + this.elementTimestampOrTimerFireTimestamp = elementTimestampOrTimerFireTimestamp; + this.boundedWindow = boundedWindow; + this.paneInfo = paneInfo; + + TimerFamilyDeclaration timerFamilyDeclaration = + doFnSignature.timerFamilyDeclarations().get(timerFamilyId); + this.timeDomain = + DoFnSignatures.getTimerFamilySpecOrThrow(timerFamilyDeclaration, doFn).getTimeDomain(); + } @Override - public void set(String timerId, Instant absoluteTime) {} + public void set(String dynamicTimerTag, Instant absoluteTime) { + get(dynamicTimerTag).set(absoluteTime); + } @Override - public org.apache.beam.sdk.state.Timer get(String timerId) { - return null; + public org.apache.beam.sdk.state.Timer get(String dynamicTimerTag) { + return new FnApiTimer( + timerFamilyId, + userKey, + dynamicTimerTag, + boundedWindow, + elementTimestampOrTimerHoldTimestamp, + elementTimestampOrTimerFireTimestamp, + paneInfo, + timeDomain); } } @@ -1447,6 +1500,9 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { // For the initial timestamps we pass in the current elements timestamp for the hold timestamp // and the current element's timestamp which will be used for the fire timestamp if this // timer is in the EVENT time domain. + TimerDeclaration timerDeclaration = doFnSignature.timerDeclarations().get(timerId); + TimeDomain timeDomain = + DoFnSignatures.getTimerSpecOrThrow(timerDeclaration, doFn).getTimeDomain(); return new FnApiTimer( timerId, ((KV) currentElement.getValue()).getKey(), @@ -1454,13 +1510,19 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { currentWindow, currentElement.getTimestamp(), currentElement.getTimestamp(), - currentElement.getPane()); + currentElement.getPane(), + timeDomain); } @Override - public TimerMap timerFamily(String tagId) { - // TODO: implement timerFamily - return null; + public TimerMap timerFamily(String timerFamilyId) { + return new FnApiTimerMap( + timerFamilyId, + ((KV) currentElement.getValue()).getKey(), + currentWindow, + currentElement.getTimestamp(), + currentElement.getTimestamp(), + currentElement.getPane()); } @Override @@ -1689,13 +1751,25 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { currentWindow, currentTimer.getHoldTimestamp(), currentTimer.getFireTimestamp(), + currentTimer.getPane(), + currentTimeDomain); + } + + @Override + public TimerMap timerFamily(String timerFamilyId) { + return new FnApiTimerMap( + timerFamilyId, + currentTimer.getUserKey(), + currentWindow, + currentTimer.getHoldTimestamp(), + currentTimer.getFireTimestamp(), currentTimer.getPane()); } @Override - public TimerMap timerFamily(String tagId) { - // TODO: implement timerFamily - return super.timerFamily(tagId); + public String timerId(DoFn doFn) { + // Timer id is aliased to dynamic timer tag in a TimerFamily timer. + return currentTimer.getDynamicTimerTag(); } @Override From 48c2c584cf207c68509dc8db0179d97fd0f4ba3c Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Tue, 19 May 2020 22:46:44 -0700 Subject: [PATCH 022/151] Clarifies Dataflow execution environment model --- .../apache_beam/runners/dataflow/internal/apiclient.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index 06bb26029aeb..8c906bd39715 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -314,8 +314,15 @@ def __init__( if container_image_url in already_added_containers: # Do not add the pipeline environment again. + # Currently, Dataflow uses Docker container images to uniquely + # identify execution environments. Hence Dataflow executes all + # transforms that specifies the same Docker contaienr image in the + # same container. Dependencies of all environments that specifies a + # given container image will be staged in the container for that + # particular container image. # TODO(BEAM-9455): loosen this restriction to support multiple - # environments with the same container name. + # environments with the same container image when Dataflow supports + # environment specific artifact provisioning. continue already_added_containers.append(container_image_url) From 13568c83ac7155000e7f8a1241fa3f02d8acf819 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Wed, 20 May 2020 11:59:00 -0400 Subject: [PATCH 023/151] Fix grammar / spelling --- website/www/site/static/js/language-switch.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/static/js/language-switch.js b/website/www/site/static/js/language-switch.js index 57d8f20341c0..7b6cb5942117 100644 --- a/website/www/site/static/js/language-switch.js +++ b/website/www/site/static/js/language-switch.js @@ -68,7 +68,7 @@ $(document).ready(function() { }, /** * @desc Search next sibling and if it's also a code block, then store - it's type and move onto the next element. It will keep + its type and move on to the next element. It will keep looking until there is no direct code block descendant left. * @param object $el - jQuery object, from where to start searching. * @param array $lang - list to hold types, found while searching. From 7bdd52ea42aea939be59d2d38bf543f9b1710550 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Wed, 20 May 2020 12:05:12 -0400 Subject: [PATCH 024/151] Remove curly quotes from documentation code samples --- .../en/documentation/programming-guide.md | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index c30578a0ec86..5f18e2ef0202 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -2362,7 +2362,7 @@ This logical type allows creating an enumeration type consisting of a set of nam {{< highlight java >}} Schema schema = Schema.builder() … - .addLogicalTypeField(“color”, EnumerationType.create(“RED”, “GREEN”, “BLUE”)) + .addLogicalTypeField("color", EnumerationType.create("RED", "GREEN", "BLUE")) .build(); {{< /highlight >}} @@ -2370,15 +2370,15 @@ The value of this field is stored in the row as an INT32 type, however the logic you access the enumeration either as a string or a value. For example: {{< highlight java >}} -EnumerationType.Value enumValue = enumType.valueOf(“RED”); +EnumerationType.Value enumValue = enumType.valueOf("RED"); enumValue.getValue(); // Returns 0, the integer value of the constant. -enumValue.toString(); // Returns “RED”, the string value of the constant +enumValue.toString(); // Returns "RED", the string value of the constant {{< /highlight >}} Given a row object with an enumeration field, you can also extract the field as the enumeration value. {{< highlight java >}} -EnumerationType.Value enumValue = row.getLogicalTypeValue(“color”, EnumerationType.Value.class); +EnumerationType.Value enumValue = row.getLogicalTypeValue("color", EnumerationType.Value.class); {{< /highlight >}} Automatic schema inference from Java POJOs and JavaBeans automatically converts Java enums to EnumerationType logical @@ -2391,10 +2391,10 @@ OneOfType allows creating a disjoint union type over a set of schema fields. For {{< highlight java >}} Schema schema = Schema.builder() … - .addLogicalTypeField(“oneOfField”, - OneOfType.create(Field.of(“intField”, FieldType.INT32), - Field.of(“stringField”, FieldType.STRING), - Field.of(“bytesField”, FieldType.BYTES))) + .addLogicalTypeField("oneOfField", + OneOfType.create(Field.of("intField", FieldType.INT32), + Field.of("stringField", FieldType.STRING), + Field.of("bytesField", FieldType.BYTES))) .build(); {{< /highlight >}} @@ -2405,19 +2405,19 @@ logical type however defines a Value object that contains an enumeration value i {{< highlight java >}} // Returns an enumeration indicating all possible case values for the enum. // For the above example, this will be -// EnumerationType.create(“intField”, “stringField”, “bytesField”); +// EnumerationType.create("intField", "stringField", "bytesField"); EnumerationType oneOfEnum = onOfType.getCaseEnumType(); // Creates an instance of the union with the string field set. -OneOfType.Value oneOfValue = oneOfType.createValue(“stringField”, “foobar”); +OneOfType.Value oneOfValue = oneOfType.createValue("stringField", "foobar"); // Handle the oneof switch (oneOfValue.getCaseEnumType().toString()) { - case “intField”: + case "intField": return processInt(oneOfValue.getValue(Integer.class)); - case “stringField”: + case "stringField": return processString(oneOfValue.getValue(String.class)); - case “bytesField”: + case "bytesField": return processBytes(oneOfValue.getValue(bytes[].class)); } {{< /highlight >}} @@ -2565,7 +2565,7 @@ In order to select a field at the top level of a schema, the name of the field i the user ids from a `PCollection` of purchases one would write (using the `Select` transform) {{< highlight java >}} -purchases.apply(Select.fieldNames(“userId”)); +purchases.apply(Select.fieldNames("userId")); {{< /highlight >}} ##### **Nested fields** @@ -2574,7 +2574,7 @@ Individual nested fields can be specified using the dot operator. For example, t shipping address one would write {{< highlight java >}} -purchases.apply(Select.fieldNames(“shippingAddress.postCode”)); +purchases.apply(Select.fieldNames("shippingAddress.postCode")); {{< /highlight >}} ##### **Wildcards** @@ -2583,7 +2583,7 @@ The * operator can be specified at any nesting level to represent all fields at shipping-address fields one would write {{< highlight java >}} -purchases.apply(Select.fieldNames(“shippingAddress.*”)); +purchases.apply(Select.fieldNames("shippingAddress.*")); {{< /highlight >}} ##### **Arrays** @@ -2592,7 +2592,7 @@ An array field, where the array element type is a row, can also have subfields o selected, the result is an array of the selected subfield type. For example {{< highlight java >}} -purchases.apply(Select.fieldNames(“transactions[].bank”)); +purchases.apply(Select.fieldNames("transactions[].bank")); {{< /highlight >}} Will result in a row containing an array field with element-type string, containing the list of banks for each @@ -2631,7 +2631,7 @@ specific keys from the map. For example, given the following schema: The following {{< highlight java >}} -purchasesByType.apply(Select.fieldNames(“purchases{}.userId”)); +purchasesByType.apply(Select.fieldNames("purchases{}.userId")); {{< /highlight >}} Will result in a row containing an map field with key-type string and value-type string. The selected map will contain @@ -2655,7 +2655,7 @@ field as a top-level field. Both top-level and nested fields can be selected. Fo could select only the userId and streetAddress fields as follows {{< highlight java >}} -purchases.apply(Select.fieldNames(“userId”, shippingAddress.streetAddress”)); +purchases.apply(Select.fieldNames("userId", shippingAddress.streetAddress")); {{< /highlight >}} The resulting `PCollection` will have the following schema @@ -2683,7 +2683,7 @@ The resulting `PCollection` will have the following schema The same is true for wildcard selections. The following {{< highlight java >}} -purchases.apply(Select.fieldNames(“userId”, shippingAddress.*”)); +purchases.apply(Select.fieldNames("userId", shippingAddress.*")); {{< /highlight >}} Will result in the following schema @@ -2729,7 +2729,7 @@ top-level field in the resulting row. This means that if multiple fields are sel selected field will appear as its own array field. For example {{< highlight java >}} -purchases.apply(Select.fieldNames( “transactions.bank”, transactions.purchaseAmount”)); +purchases.apply(Select.fieldNames( "transactions.bank", transactions.purchaseAmount")); {{< /highlight >}} Will result in the following schema @@ -2832,7 +2832,7 @@ The simplest usage of `Group` specifies no aggregations, in which case all input are grouped together into an `ITERABLE` field. For example {{< highlight java >}} -purchases.apply(Group.byFieldNames(“userId”, shippingAddress.streetAddress”)); +purchases.apply(Group.byFieldNames("userId", shippingAddress.streetAddress")); {{< /highlight >}} The output schema of this is: @@ -2863,9 +2863,9 @@ The names of the key and values fields in the output schema can be controlled us builders, as follows: {{< highlight java >}} -purchases.apply(Group.byFieldNames(“userId”, shippingAddress.streetAddress”) - .withKeyField(“userAndStreet”) - .withValueField(“matchingPurchases”)); +purchases.apply(Group.byFieldNames("userId", shippingAddress.streetAddress") + .withKeyField("userAndStreet") + .withValueField("matchingPurchases")); {{< /highlight >}} It is quite common to apply one or more aggregations to the grouped result. Each aggregation can specify one or more fields @@ -2874,10 +2874,10 @@ following application computes three aggregations grouped by userId, with all ag output schema: {{< highlight java >}} -purchases.apply(Group.byFieldNames(“userId”) - .aggregateField(“itemId”, Count.combineFn(), “numPurchases”) - .aggregateField(“costCents”, Sum.ofLongs(), “totalSpendCents”) - .aggregateField(“costCents”, Top.largestLongsFn(10), “topPurchases”)); +purchases.apply(Group.byFieldNames("userId") + .aggregateField("itemId", Count.combineFn(), "numPurchases") + .aggregateField("costCents", Sum.ofLongs(), "totalSpendCents") + .aggregateField("costCents", Top.largestLongsFn(10), "topPurchases")); {{< /highlight >}} The result of this aggregation will have the following schema: @@ -2915,7 +2915,7 @@ and is specified with the `using` keyword: PCollection transactions = readTransactions(); PCollection reviews = readReviews(); PCollection joined = transactions.apply( - Join.innerJoin(reviews).using(“userId”, “productId”)); + Join.innerJoin(reviews).using("userId", "productId")); {{< /highlight >}} The resulting schema is the following: @@ -2948,8 +2948,8 @@ Review schema named those fields differently than the Transaction schema, then w PCollection joined = transactions.apply( Join.innerJoin(reviews).on( FieldsEqual - .left(“userId”, “productId”) - .right(“reviewUserId”, “reviewProductId”))); + .left("userId", "productId") + .right("reviewUserId", "reviewProductId"))); {{< /highlight >}} In addition to inner joins, the Join transform supports full outer joins, left outer joins, and right outer joins. @@ -2971,8 +2971,8 @@ which all predicates return true will pass the filter. For example the following {{< highlight java >}} purchases.apply(Filter - .whereFieldName(“costCents”, c -> c > 100 * 20) - .whereFieldName(“shippingAddress.country”, c -> c.equals(“de”)); + .whereFieldName("costCents", c -> c > 100 * 20) + .whereFieldName("shippingAddress.country", c -> c.equals("de")); {{< /highlight >}} Will produce all purchases made from Germany with a purchase price of greater than twenty cents. @@ -2989,9 +2989,9 @@ For example, the following application {{< highlight java >}} purchases.apply(AddFields.create() - .field(“timeOfDaySeconds”, FieldType.INT32) - .field(“shippingAddress.deliveryNotes”, FieldType.STRING) - .field(“transactions.isFlagged, FieldType.BOOLEAN, false)); + .field("timeOfDaySeconds", FieldType.INT32) + .field("shippingAddress.deliveryNotes", FieldType.STRING) + .field("transactions.isFlagged, FieldType.BOOLEAN, false)); {{< /highlight >}} Results in a `PCollection` with an expanded schema. All of the rows and fields of the input, but also with the specified @@ -3007,7 +3007,7 @@ syntax. For example, the following snippet {{< highlight java >}} -purchases.apply(DropFields.fields(“userId”, “shippingAddress.streetAddress”)); +purchases.apply(DropFields.fields("userId", "shippingAddress.streetAddress")); {{< /highlight >}} Results in a copy of the input with those two fields and their corresponding values removed. @@ -3024,8 +3024,8 @@ For example, the following snippet {{< highlight java >}} purchases.apply(RenameFields.create() - .rename(“userId”, “userIdentifier”) - .rename(“shippingAddress.streetAddress”, “shippingAddress.street”)); + .rename("userId", "userIdentifier") + .rename("shippingAddress.streetAddress", "shippingAddress.street")); {{< /highlight >}} Results in the same set of unmodified input elements, however the schema on the PCollection has been changed to rename @@ -3113,7 +3113,7 @@ using the above-described selection expressions, as follows: {{< highlight java >}} purchases.appy(ParDo.of(new DoFn() { @ProcessElement public void process( - @FieldAccess(“userId”) String userId, @FieldAccess(“itemId”) long itemId) { + @FieldAccess("userId") String userId, @FieldAccess("itemId") long itemId) { ... } })); @@ -3124,7 +3124,7 @@ You can also select nested fields, as follows. {{< highlight java >}} purchases.appy(ParDo.of(new DoFn() { @ProcessElement public void process( - @FieldAccess(“shippingAddress.street”) String street) { + @FieldAccess("shippingAddress.street") String street) { ... } })); @@ -3953,8 +3953,8 @@ It is important to note that if, for example, you specify AfterCount(50) and only 32 elements arrive, those 32 elements sit around forever. If the 32 elements are important to you, consider using [composite triggers](#composite-triggers) to combine multiple -conditions. This allows you to specify multiple firing conditions such as “fire -either when I receive 50 elements, or every 1 second”. +conditions. This allows you to specify multiple firing conditions such as "fire +either when I receive 50 elements, or every 1 second". ### 9.4. Setting a trigger {#setting-a-trigger} From 696ebf29c8235c12a3fae3ce11ce2f512fad244a Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Wed, 20 May 2020 09:57:27 -0700 Subject: [PATCH 025/151] Fixes typos. --- .../python/apache_beam/runners/dataflow/internal/apiclient.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index 8c906bd39715..d0e7911264ed 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -316,8 +316,8 @@ def __init__( # Currently, Dataflow uses Docker container images to uniquely # identify execution environments. Hence Dataflow executes all - # transforms that specifies the same Docker contaienr image in the - # same container. Dependencies of all environments that specifies a + # transforms that specify the same Docker container image in the + # same container. Dependencies of all environments that specify a # given container image will be staged in the container for that # particular container image. # TODO(BEAM-9455): loosen this restriction to support multiple From cdc78211282be7c9b9b794996388b6e0428d3ba9 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 20 May 2020 13:17:25 -0400 Subject: [PATCH 026/151] Update test list. --- release/src/main/scripts/mass_comment.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index bd9e26aa3789..b418aa3c9176 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -29,23 +29,26 @@ COMMENTS_TO_ADD=[ + "Run Release Gradle Build", "Run Go PostCommit", "Run Java PostCommit", - "Run Java PortabilityApi PostCommit", "Run Java Flink PortableValidatesRunner Batch", "Run Java Flink PortableValidatesRunner Streaming", "Run Apex ValidatesRunner", "Run Dataflow ValidatesRunner", "Run Flink ValidatesRunner", "Run Gearpump ValidatesRunner", - "Run Dataflow PortabilityApi ValidatesRunner", "Run Samza ValidatesRunner", "Run Spark ValidatesRunner", + "Run Java Spark PortableValidatesRunner Batch", + "Run Python Spark ValidatesRunner", "Run Python Dataflow ValidatesContainer", "Run Python Dataflow ValidatesRunner", "Run Python 3.5 Flink ValidatesRunner", "Run Python 2 PostCommit", "Run Python 3.5 PostCommit", + "Run Python 3.6 PostCommit", + "Run Python 3.7 PostCommit", "Run SQL PostCommit", "Run Go PreCommit", "Run Java PreCommit", From 0a8d955e49776b872e8fda59cbeb0870afa68784 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 20 May 2020 14:23:02 -0400 Subject: [PATCH 027/151] Update verify_release_build.sh to use mass_comment.py. --- .../src/main/scripts/verify_release_build.sh | 39 +------------------ 1 file changed, 1 insertion(+), 38 deletions(-) diff --git a/release/src/main/scripts/verify_release_build.sh b/release/src/main/scripts/verify_release_build.sh index bc3671f46d93..77c184bc463a 100755 --- a/release/src/main/scripts/verify_release_build.sh +++ b/release/src/main/scripts/verify_release_build.sh @@ -40,40 +40,6 @@ BEAM_REPO_URL=https://github.com/apache/beam.git RELEASE_BRANCH=release-${RELEASE_VER} WORKING_BRANCH=postcommit_validation_pr -JOB_TRIGGER_PHRASES=( - # To verify Gradle release build - "**Run Release Gradle Build**" - # To run all PostCommit jobs - "Run Go PostCommit" - "Run Java PostCommit" - "Run Java PortabilityApi PostCommit" - "Run Java Flink PortableValidatesRunner Batch" - "Run Java Flink PortableValidatesRunner Streaming" - "Run Apex ValidatesRunner" - "Run Dataflow ValidatesRunner" - "Run Flink ValidatesRunner" - "Run Gearpump ValidatesRunner" - "Run Dataflow PortabilityApi ValidatesRunner" - "Run Samza ValidatesRunner" - "Run Spark ValidatesRunner" - "Run Python Dataflow ValidatesContainer" - "Run Python Dataflow ValidatesRunner" - "Run Python 3.5 Flink ValidatesRunner" - # Python versions match those in run_rc_validation.sh. - "Run Python 2 PostCommit" - "Run Python 3.5 PostCommit" - "Run SQL PostCommit" - "Run Go PreCommit" - "Run Java PreCommit" - "Run Java_Examples_Dataflow PreCommit" - "Run JavaPortabilityApi PreCommit" - "Run Portable_Python PreCommit" - "Run PythonLint PreCommit" - "Run Python PreCommit" - "Run Python DockerBuild PreCommit" -) - - function clean_up(){ echo "" echo "==================== Final Cleanup ====================" @@ -164,12 +130,9 @@ if [[ ! -z `which hub` ]]; then git commit -m "Changed version.py and gradle.properties to python dev version to create a test PR" --quiet git push -f ${GITHUB_USERNAME} --quiet - trigger_phrases=$(IFS=$'\n'; echo "${JOB_TRIGGER_PHRASES[*]}") hub pull-request -b apache:${RELEASE_BRANCH} -h ${GITHUB_USERNAME}:${WORKING_BRANCH} -F- <<<"[DO NOT MERGE] Run all PostCommit and PreCommit Tests against Release Branch - Please comment as instructions below, one phrase per comment: - - ${trigger_phrases}" + You can run many tests automatically using release/src/main/scripts/mass_comment.py." echo "" echo "[NOTE]: Please make sure all test targets have been invoked." From 84343b06fa63aec081ea2c16b2ec3ebd6baad394 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 20 May 2020 14:35:18 -0400 Subject: [PATCH 028/151] Run yapf on mass_comment.py. --- release/src/main/scripts/mass_comment.py | 69 ++++++++++++------------ 1 file changed, 36 insertions(+), 33 deletions(-) diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index b418aa3c9176..b0f74896dc22 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -27,39 +27,39 @@ import requests from datetime import datetime - -COMMENTS_TO_ADD=[ - "Run Release Gradle Build", - "Run Go PostCommit", - "Run Java PostCommit", - "Run Java Flink PortableValidatesRunner Batch", - "Run Java Flink PortableValidatesRunner Streaming", - "Run Apex ValidatesRunner", - "Run Dataflow ValidatesRunner", - "Run Flink ValidatesRunner", - "Run Gearpump ValidatesRunner", - "Run Samza ValidatesRunner", - "Run Spark ValidatesRunner", - "Run Java Spark PortableValidatesRunner Batch", - "Run Python Spark ValidatesRunner", - "Run Python Dataflow ValidatesContainer", - "Run Python Dataflow ValidatesRunner", - "Run Python 3.5 Flink ValidatesRunner", - "Run Python 2 PostCommit", - "Run Python 3.5 PostCommit", - "Run Python 3.6 PostCommit", - "Run Python 3.7 PostCommit", - "Run SQL PostCommit", - "Run Go PreCommit", - "Run Java PreCommit", - "Run Java_Examples_Dataflow PreCommit", - "Run JavaPortabilityApi PreCommit", - "Run Portable_Python PreCommit", - "Run PythonLint PreCommit", - "Run Python PreCommit", - "Run Python DockerBuild PreCommit" +COMMENTS_TO_ADD = [ + "Run Release Gradle Build", + "Run Go PostCommit", + "Run Java PostCommit", + "Run Java Flink PortableValidatesRunner Batch", + "Run Java Flink PortableValidatesRunner Streaming", + "Run Apex ValidatesRunner", + "Run Dataflow ValidatesRunner", + "Run Flink ValidatesRunner", + "Run Gearpump ValidatesRunner", + "Run Samza ValidatesRunner", + "Run Spark ValidatesRunner", + "Run Java Spark PortableValidatesRunner Batch", + "Run Python Spark ValidatesRunner", + "Run Python Dataflow ValidatesContainer", + "Run Python Dataflow ValidatesRunner", + "Run Python 3.5 Flink ValidatesRunner", + "Run Python 2 PostCommit", + "Run Python 3.5 PostCommit", + "Run Python 3.6 PostCommit", + "Run Python 3.7 PostCommit", + "Run SQL PostCommit", + "Run Go PreCommit", + "Run Java PreCommit", + "Run Java_Examples_Dataflow PreCommit", + "Run JavaPortabilityApi PreCommit", + "Run Portable_Python PreCommit", + "Run PythonLint PreCommit", + "Run Python PreCommit", + "Run Python DockerBuild PreCommit" ] + def executeGHGraphqlQuery(accessToken, query): '''Runs graphql query on GitHub.''' url = 'https://api.github.com/graphql' @@ -67,6 +67,7 @@ def executeGHGraphqlQuery(accessToken, query): r = requests.post(url=url, json={'query': query}, headers=headers) return r.json() + def getSubjectId(accessToken, prNumber): query = ''' query FindPullRequestID { @@ -80,6 +81,7 @@ def getSubjectId(accessToken, prNumber): response = executeGHGraphqlQuery(accessToken, query) return response['data']['repository']['pullRequest']['id'] + def fetchGHData(accessToken, subjectId, commentBody): '''Fetches GitHub data required for reporting Beam metrics''' query = ''' @@ -99,6 +101,7 @@ def fetchGHData(accessToken, subjectId, commentBody): ''' % (subjectId, commentBody) return executeGHGraphqlQuery(accessToken, query) + def postComments(accessToken, subjectId): ''' Main workhorse method. Fetches data from GitHub and puts it in metrics table. @@ -108,6 +111,7 @@ def postComments(accessToken, subjectId): jsonData = fetchGHData(accessToken, subjectId, commentBody) print(jsonData) + def probeGitHubIsUp(): ''' Returns True if GitHub responds to simple queries. Else returns False. @@ -126,10 +130,9 @@ def probeGitHubIsUp(): ''' print("Started.") - if not probeGitHubIsUp(): print("GitHub is unavailable, skipping fetching data.") - exit(); + exit() print("GitHub is available start fetching data.") From e88362ee2cc56f5c58a11fa4e36edbfabb83eb67 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 20 May 2020 11:01:53 -0700 Subject: [PATCH 029/151] Skip checking input determinism for GBKs embedded in x-lang transforms. We can assume safely assume that the expansion service constructs transforms correctly. Change-Id: I46d40e40ac33933004836681d06f91c031655f06 --- .../python/apache_beam/runners/dataflow/dataflow_runner.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 4c02168cee58..505316d5fdf6 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -849,8 +849,15 @@ def _verify_gbk_coders(self, transform, pcoll): # # TODO(ccy): make Coder inference and checking less specialized and more # comprehensive. + parent = pcoll.producer if parent: + # Skip the check because we can assume that any x-lang transform is + # properly formed (the onus is on the expansion service to construct + # transforms correctly). + if isinstance(parent.transform, RunnerAPIPTransformHolder): + return + coder = parent.transform._infer_output_coder() # pylint: disable=protected-access if not coder: coder = self._get_coder(pcoll.element_type or typehints.Any, None) From d5d7c88b2d330f02260cab3b1075509a34181f3a Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 20 May 2020 14:50:41 -0400 Subject: [PATCH 030/151] Remove irrelevant instruction. --- website/www/site/content/en/contribute/release-guide.md | 1 - 1 file changed, 1 deletion(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 55ec1d126d9a..7e199ae9d5ab 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -406,7 +406,6 @@ You can use [mass_comment.py](https://github.com/apache/beam/blob/master/release 1. Check the build result. 2. If build failed, scan log will contain all failures. 3. You should stabilize the release branch until release build succeeded. - 4. The script will output a set of Jenkins phrases to enter in the created PR. There are some projects that don't produce the artifacts, e.g. `beam-test-tools`, you may be able to ignore failures there. From 583991565db7b684957c9a97a0ed6bb09593c01f Mon Sep 17 00:00:00 2001 From: Yichi Zhang Date: Wed, 20 May 2020 11:56:36 -0700 Subject: [PATCH 031/151] Address comments and add tests --- .../beam/fn/harness/FnApiDoFnRunner.java | 50 ++-- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 245 +++++++++++++----- 2 files changed, 215 insertions(+), 80 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 7ee066956de6..ab07ca319ac0 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -92,7 +92,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.StateDeclaration; -import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerDeclaration; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerFamilyDeclaration; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; @@ -461,19 +460,10 @@ public void accept(WindowedValue input) throws Exception { // Extract out relevant TimerFamilySpec information in preparation for execution. for (Map.Entry entry : parDoPayload.getTimerFamilySpecsMap().entrySet()) { + // The timer family spec map key is either from timerId of timer declaration or + // timerFamilyId from timer family declaration. String timerIdOrTimerFamilyId = entry.getKey(); - TimeDomain timeDomain; - if (timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX)) { - timeDomain = - DoFnSignatures.getTimerFamilySpecOrThrow( - doFnSignature.timerFamilyDeclarations().get(timerIdOrTimerFamilyId), doFn) - .getTimeDomain(); - } else { - timeDomain = - DoFnSignatures.getTimerSpecOrThrow( - doFnSignature.timerDeclarations().get(timerIdOrTimerFamilyId), doFn) - .getTimeDomain(); - } + TimeDomain timeDomain = translateTimeDomain(entry.getValue().getTimeDomain()); Coder> timerCoder = (Coder) rehydratedComponents.getCoder(entry.getValue().getTimerFamilyCoderId()); timerFamilyInfosBuilder.put(timerIdOrTimerFamilyId, KV.of(timeDomain, timerCoder)); @@ -976,6 +966,8 @@ private void processTimer( currentTimer = timer; currentTimeDomain = timeDomain; onTimerContext = new OnTimerContext<>(timer.getUserKey()); + // The timerIdOrTimerFamilyId contains either a timerId from timer declaration or timerFamilyId + // from timer family declaration. String timerId = timerIdOrTimerFamilyId.startsWith(TimerFamilyDeclaration.PREFIX) ? "" @@ -1057,6 +1049,9 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { Instant elementTimestampOrTimerFireTimestamp, PaneInfo paneInfo, TimeDomain timeDomain) { + if (timerId.equals("ts-event") && timeDomain.equals(TimeDomain.PROCESSING_TIME)) { + throw new IllegalStateException("illegal"); + } this.timerId = timerId; this.userKey = userKey; this.dynamicTimerTag = dynamicTimerTag; @@ -1246,11 +1241,9 @@ private class FnApiTimerMap implements TimerMap { this.elementTimestampOrTimerFireTimestamp = elementTimestampOrTimerFireTimestamp; this.boundedWindow = boundedWindow; this.paneInfo = paneInfo; - - TimerFamilyDeclaration timerFamilyDeclaration = - doFnSignature.timerFamilyDeclarations().get(timerFamilyId); this.timeDomain = - DoFnSignatures.getTimerFamilySpecOrThrow(timerFamilyDeclaration, doFn).getTimeDomain(); + translateTimeDomain( + parDoPayload.getTimerFamilySpecsMap().get(timerFamilyId).getTimeDomain()); } @Override @@ -1500,9 +1493,8 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { // For the initial timestamps we pass in the current elements timestamp for the hold timestamp // and the current element's timestamp which will be used for the fire timestamp if this // timer is in the EVENT time domain. - TimerDeclaration timerDeclaration = doFnSignature.timerDeclarations().get(timerId); TimeDomain timeDomain = - DoFnSignatures.getTimerSpecOrThrow(timerDeclaration, doFn).getTimeDomain(); + translateTimeDomain(parDoPayload.getTimerFamilySpecsMap().get(timerId).getTimeDomain()); return new FnApiTimer( timerId, ((KV) currentElement.getValue()).getKey(), @@ -1744,15 +1736,17 @@ public State state(String stateId, boolean alwaysFetched) { @Override public org.apache.beam.sdk.state.Timer timer(String timerId) { + TimeDomain timeDomain = + translateTimeDomain(parDoPayload.getTimerFamilySpecsMap().get(timerId).getTimeDomain()); return new FnApiTimer( timerId, currentTimer.getUserKey(), - currentTimer.getDynamicTimerTag(), + "", currentWindow, currentTimer.getHoldTimestamp(), currentTimer.getFireTimestamp(), currentTimer.getPane(), - currentTimeDomain); + timeDomain); } @Override @@ -1782,4 +1776,18 @@ public String getErrorContext() { return "FnApiDoFnRunner/OnTimer"; } } + + private TimeDomain translateTimeDomain( + org.apache.beam.model.pipeline.v1.RunnerApi.TimeDomain.Enum domain) { + switch (domain) { + case EVENT_TIME: + return TimeDomain.EVENT_TIME; + case PROCESSING_TIME: + return TimeDomain.PROCESSING_TIME; + case SYNCHRONIZED_PROCESSING_TIME: + return TimeDomain.SYNCHRONIZED_PROCESSING_TIME; + default: + throw new IllegalArgumentException("Unknown time domain"); + } + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 1da967269b32..e763fface9f4 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -94,6 +94,7 @@ import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerMap; import org.apache.beam.sdk.state.TimerSpec; import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.state.ValueState; @@ -759,58 +760,6 @@ public void testUsingMetrics() throws Exception { assertThat(result, containsInAnyOrder(expected.toArray())); } - private static class TestTimerfulDoFn extends DoFn, String> { - @StateId("bag") - private final StateSpec> bagStateSpec = StateSpecs.bag(StringUtf8Coder.of()); - - @TimerId("event") - private final TimerSpec eventTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); - - @TimerId("processing") - private final TimerSpec processingTimerSpec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); - - @ProcessElement - public void processElement( - ProcessContext context, - @StateId("bag") BagState bagState, - @TimerId("event") Timer eventTimeTimer, - @TimerId("processing") Timer processingTimeTimer) { - context.output("main" + context.element().getKey() + Iterables.toString(bagState.read())); - bagState.add(context.element().getValue()); - eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(1L)); - processingTimeTimer.offset(Duration.millis(2L)); - processingTimeTimer.setRelative(); - } - - @OnTimer("event") - public void eventTimer( - OnTimerContext context, - @StateId("bag") BagState bagState, - @TimerId("event") Timer eventTimeTimer, - @TimerId("processing") Timer processingTimeTimer) { - context.output("event" + Iterables.toString(bagState.read())); - bagState.add("event"); - eventTimeTimer - .withOutputTimestamp(context.timestamp()) - .set(context.fireTimestamp().plus(11L)); - processingTimeTimer.offset(Duration.millis(12L)); - processingTimeTimer.setRelative(); - } - - @OnTimer("processing") - public void processingTimer( - OnTimerContext context, - @StateId("bag") BagState bagState, - @TimerId("event") Timer eventTimeTimer, - @TimerId("processing") Timer processingTimeTimer) { - context.output("processing" + Iterables.toString(bagState.read())); - bagState.add("processing"); - eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(21L)); - processingTimeTimer.offset(Duration.millis(22L)); - processingTimeTimer.setRelative(); - } - } - @Test public void testTimers() throws Exception { dateTimeProvider.setDateTimeFixed(10000L); @@ -889,6 +838,10 @@ public void testTimers() throws Exception { LogicalEndpoint eventTimer = LogicalEndpoint.timer("57L", TEST_TRANSFORM_ID, "ts-event"); LogicalEndpoint processingTimer = LogicalEndpoint.timer("57L", TEST_TRANSFORM_ID, "ts-processing"); + LogicalEndpoint eventFamilyTimer = + LogicalEndpoint.timer("57L", TEST_TRANSFORM_ID, "tfs-event-family"); + LogicalEndpoint processingFamilyTimer = + LogicalEndpoint.timer("57L", TEST_TRANSFORM_ID, "tfs-processing-family"); // Ensure that bag user state that is initially empty or populated works. // Ensure that the key order does not matter when we traverse over KV pairs. FnDataReceiver> mainInput = @@ -909,6 +862,12 @@ public void testTimers() throws Exception { processingTimer, timerInGlobalWindow("C", new Instant(1800L), new Instant(2800L))); fakeTimerClient.sendTimer( processingTimer, timerInGlobalWindow("B", new Instant(1900L), new Instant(2900L))); + fakeTimerClient.sendTimer( + eventFamilyTimer, + timerInGlobalWindow("B", "event-timer2", new Instant(2000L), new Instant(3000L))); + fakeTimerClient.sendTimer( + processingFamilyTimer, + timerInGlobalWindow("Y", "processing-timer2", new Instant(2100L), new Instant(3100L))); assertThat( mainOutputValues, contains( @@ -921,7 +880,9 @@ public void testTimers() throws Exception { timestampedValueInGlobalWindow("event[A0, event]", new Instant(1600L)), timestampedValueInGlobalWindow("processing[X0, X1, X2]", new Instant(1700L)), timestampedValueInGlobalWindow("processing[C0]", new Instant(1800L)), - timestampedValueInGlobalWindow("processing[event]", new Instant(1900L)))); + timestampedValueInGlobalWindow("processing[event]", new Instant(1900L)), + timestampedValueInGlobalWindow("event-family[event, processing]", new Instant(2000L)), + timestampedValueInGlobalWindow("processing-family[Y1, Y2]", new Instant(2100L)))); assertThat( fakeTimerClient.getTimers(eventTimer), contains( @@ -934,7 +895,9 @@ public void testTimers() throws Exception { timerInGlobalWindow("A", new Instant(1600L), new Instant(2611L)), timerInGlobalWindow("X", new Instant(1700L), new Instant(1721L)), timerInGlobalWindow("C", new Instant(1800L), new Instant(1821L)), - timerInGlobalWindow("B", new Instant(1900L), new Instant(1921L)))); + timerInGlobalWindow("B", new Instant(1900L), new Instant(1921L)), + timerInGlobalWindow("B", new Instant(2000L), new Instant(2031L)), + timerInGlobalWindow("Y", new Instant(2100L), new Instant(2141L)))); assertThat( fakeTimerClient.getTimers(processingTimer), contains( @@ -947,19 +910,58 @@ public void testTimers() throws Exception { timerInGlobalWindow("A", new Instant(1600L), new Instant(10012L)), timerInGlobalWindow("X", new Instant(1700L), new Instant(10022L)), timerInGlobalWindow("C", new Instant(1800L), new Instant(10022L)), - timerInGlobalWindow("B", new Instant(1900L), new Instant(10022L)))); + timerInGlobalWindow("B", new Instant(1900L), new Instant(10022L)), + timerInGlobalWindow("B", new Instant(2000L), new Instant(10032L)), + timerInGlobalWindow("Y", new Instant(2100L), new Instant(10042L)))); + assertThat( + fakeTimerClient.getTimers(eventFamilyTimer), + contains( + timerInGlobalWindow("X", "event-timer1", new Instant(1000L), new Instant(1003L)), + timerInGlobalWindow("Y", "event-timer1", new Instant(1100L), new Instant(1103L)), + timerInGlobalWindow("X", "event-timer1", new Instant(1200L), new Instant(1203L)), + timerInGlobalWindow("Y", "event-timer1", new Instant(1300L), new Instant(1303L)), + timerInGlobalWindow("A", "event-timer1", new Instant(1400L), new Instant(2413L)), + timerInGlobalWindow("B", "event-timer1", new Instant(1500L), new Instant(2513L)), + timerInGlobalWindow("A", "event-timer1", new Instant(1600L), new Instant(2613L)), + timerInGlobalWindow("X", "event-timer1", new Instant(1700L), new Instant(1723L)), + timerInGlobalWindow("C", "event-timer1", new Instant(1800L), new Instant(1823L)), + timerInGlobalWindow("B", "event-timer1", new Instant(1900L), new Instant(1923L)), + timerInGlobalWindow("B", "event-timer1", new Instant(2000L), new Instant(2033L)), + timerInGlobalWindow("Y", "event-timer1", new Instant(2100L), new Instant(2143L)))); + assertThat( + fakeTimerClient.getTimers(processingFamilyTimer), + contains( + timerInGlobalWindow("X", "processing-timer1", new Instant(1000L), new Instant(10004L)), + timerInGlobalWindow("Y", "processing-timer1", new Instant(1100L), new Instant(10004L)), + timerInGlobalWindow("X", "processing-timer1", new Instant(1200L), new Instant(10004L)), + timerInGlobalWindow("Y", "processing-timer1", new Instant(1300L), new Instant(10004L)), + timerInGlobalWindow("A", "processing-timer1", new Instant(1400L), new Instant(10014L)), + timerInGlobalWindow("B", "processing-timer1", new Instant(1500L), new Instant(10014L)), + timerInGlobalWindow("A", "processing-timer1", new Instant(1600L), new Instant(10014L)), + timerInGlobalWindow("X", "processing-timer1", new Instant(1700L), new Instant(10024L)), + timerInGlobalWindow("C", "processing-timer1", new Instant(1800L), new Instant(10024L)), + timerInGlobalWindow("B", "processing-timer1", new Instant(1900L), new Instant(10024L)), + timerInGlobalWindow("B", "processing-timer1", new Instant(2000L), new Instant(10034L)), + timerInGlobalWindow( + "Y", "processing-timer1", new Instant(2100L), new Instant(10044L)))); mainOutputValues.clear(); assertFalse(fakeTimerClient.isOutboundClosed(eventTimer)); assertFalse(fakeTimerClient.isOutboundClosed(processingTimer)); + assertFalse(fakeTimerClient.isOutboundClosed(eventFamilyTimer)); + assertFalse(fakeTimerClient.isOutboundClosed(processingFamilyTimer)); fakeTimerClient.closeInbound(eventTimer); fakeTimerClient.closeInbound(processingTimer); + fakeTimerClient.closeInbound(eventFamilyTimer); + fakeTimerClient.closeInbound(processingFamilyTimer); Iterables.getOnlyElement(finishFunctionRegistry.getFunctions()).run(); assertThat(mainOutputValues, empty()); assertTrue(fakeTimerClient.isOutboundClosed(eventTimer)); assertTrue(fakeTimerClient.isOutboundClosed(processingTimer)); + assertTrue(fakeTimerClient.isOutboundClosed(eventFamilyTimer)); + assertTrue(fakeTimerClient.isOutboundClosed(processingFamilyTimer)); Iterables.getOnlyElement(teardownFunctions).run(); assertThat(mainOutputValues, empty()); @@ -967,29 +969,154 @@ public void testTimers() throws Exception { assertEquals( ImmutableMap.builder() .put(bagUserStateKey("bag", "X"), encode("X0", "X1", "X2", "processing")) - .put(bagUserStateKey("bag", "Y"), encode("Y1", "Y2")) + .put(bagUserStateKey("bag", "Y"), encode("Y1", "Y2", "processing-family")) .put(bagUserStateKey("bag", "A"), encode("A0", "event", "event")) - .put(bagUserStateKey("bag", "B"), encode("event", "processing")) + .put(bagUserStateKey("bag", "B"), encode("event", "processing", "event-family")) .put(bagUserStateKey("bag", "C"), encode("C0", "processing")) .build(), fakeStateClient.getData()); } + private org.apache.beam.runners.core.construction.Timer timerInGlobalWindow( + K userKey, Instant holdTimestamp, Instant fireTimestamp) { + return timerInGlobalWindow(userKey, "", holdTimestamp, fireTimestamp); + } + private WindowedValue valueInWindow(T value, BoundedWindow window) { return WindowedValue.of(value, window.maxTimestamp(), window, PaneInfo.NO_FIRING); } private org.apache.beam.runners.core.construction.Timer timerInGlobalWindow( - K userKey, Instant holdTimestamp, Instant fireTimestamp) { + K userKey, String dynamicTimerTag, Instant holdTimestamp, Instant fireTimestamp) { return org.apache.beam.runners.core.construction.Timer.of( userKey, - "", + dynamicTimerTag, Collections.singletonList(GlobalWindow.INSTANCE), fireTimestamp, holdTimestamp, PaneInfo.NO_FIRING); } + private static class TestTimerfulDoFn extends DoFn, String> { + @StateId("bag") + private final StateSpec> bagStateSpec = StateSpecs.bag(StringUtf8Coder.of()); + + @TimerId("event") + private final TimerSpec eventTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @TimerId("processing") + private final TimerSpec processingTimerSpec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + @TimerFamily("event-family") + private final TimerSpec eventTimerFamilySpec = TimerSpecs.timerMap(TimeDomain.EVENT_TIME); + + @TimerFamily("processing-family") + private final TimerSpec processingTimerFamilySpec = + TimerSpecs.timerMap(TimeDomain.PROCESSING_TIME); + + @ProcessElement + public void processElement( + ProcessContext context, + @StateId("bag") BagState bagState, + @TimerId("event") Timer eventTimeTimer, + @TimerId("processing") Timer processingTimeTimer, + @TimerFamily("event-family") TimerMap eventTimerFamily, + @TimerFamily("processing-family") TimerMap processingTimerFamily) { + context.output("main" + context.element().getKey() + Iterables.toString(bagState.read())); + bagState.add(context.element().getValue()); + eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(1L)); + processingTimeTimer.offset(Duration.millis(2L)); + processingTimeTimer.setRelative(); + eventTimerFamily + .get("event-timer1") + .withOutputTimestamp(context.timestamp()) + .set(context.timestamp().plus(3L)); + processingTimerFamily.get("processing-timer1").offset(Duration.millis(4L)).setRelative(); + } + + @OnTimer("event") + public void eventTimer( + OnTimerContext context, + @StateId("bag") BagState bagState, + @TimerId("event") Timer eventTimeTimer, + @TimerId("processing") Timer processingTimeTimer, + @TimerFamily("event-family") TimerMap eventTimerFamily, + @TimerFamily("processing-family") TimerMap processingTimerFamily) { + context.output("event" + Iterables.toString(bagState.read())); + bagState.add("event"); + eventTimeTimer + .withOutputTimestamp(context.timestamp()) + .set(context.fireTimestamp().plus(11L)); + processingTimeTimer.offset(Duration.millis(12L)); + processingTimeTimer.setRelative(); + eventTimerFamily + .get("event-timer1") + .withOutputTimestamp(context.timestamp()) + .set(context.fireTimestamp().plus(13L)); + processingTimerFamily.get("processing-timer1").offset(Duration.millis(14L)).setRelative(); + } + + @OnTimer("processing") + public void processingTimer( + OnTimerContext context, + @StateId("bag") BagState bagState, + @TimerId("event") Timer eventTimeTimer, + @TimerId("processing") Timer processingTimeTimer, + @TimerFamily("event-family") TimerMap eventTimerFamily, + @TimerFamily("processing-family") TimerMap processingTimerFamily) { + context.output("processing" + Iterables.toString(bagState.read())); + bagState.add("processing"); + eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(21L)); + processingTimeTimer.offset(Duration.millis(22L)); + processingTimeTimer.setRelative(); + eventTimerFamily + .get("event-timer1") + .withOutputTimestamp(context.timestamp()) + .set(context.timestamp().plus(23L)); + processingTimerFamily.get("processing-timer1").offset(Duration.millis(24L)).setRelative(); + } + + @OnTimerFamily("event-family") + public void eventFamilyOnTimer( + OnTimerContext context, + @StateId("bag") BagState bagState, + @TimerId("event") Timer eventTimeTimer, + @TimerId("processing") Timer processingTimeTimer, + @TimerFamily("event-family") TimerMap eventTimerFamily, + @TimerFamily("processing-family") TimerMap processingTimerFamily) { + context.output("event-family" + Iterables.toString(bagState.read())); + bagState.add("event-family"); + eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(31L)); + processingTimeTimer.offset(Duration.millis(32L)); + processingTimeTimer.setRelative(); + eventTimerFamily + .get("event-timer1") + .withOutputTimestamp(context.timestamp()) + .set(context.timestamp().plus(33L)); + processingTimerFamily.get("processing-timer1").offset(Duration.millis(34L)).setRelative(); + } + + @OnTimerFamily("processing-family") + public void processingFamilyOnTimer( + OnTimerContext context, + @StateId("bag") BagState bagState, + @TimerId("event") Timer eventTimeTimer, + @TimerId("processing") Timer processingTimeTimer, + @TimerFamily("event-family") TimerMap eventTimerFamily, + @TimerFamily("processing-family") TimerMap processingTimerFamily) { + context.output("processing-family" + Iterables.toString(bagState.read())); + bagState.add("processing-family"); + eventTimeTimer.withOutputTimestamp(context.timestamp()).set(context.timestamp().plus(41L)); + processingTimeTimer.offset(Duration.millis(42L)); + processingTimeTimer.setRelative(); + eventTimerFamily + .get("event-timer1") + .withOutputTimestamp(context.timestamp()) + .set(context.timestamp().plus(43L)); + processingTimerFamily.get("processing-timer1").offset(Duration.millis(44L)).setRelative(); + } + } + /** * Produces a multimap side input {@link StateKey} for the test PTransform id in the global * window. From b2fefaf9e02b83942eeb1c7b85c8f765da89a14d Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Wed, 20 May 2020 12:58:41 -0700 Subject: [PATCH 032/151] [BEAM-9679] Add Core Transforms section / GroupByKey lesson to the Go SDK katas (#11734) --- .../GroupByKey/GroupByKey/cmd/main.go | 43 +++ .../GroupByKey/GroupByKey/go.mod | 26 ++ .../GroupByKey/GroupByKey/go.sum | 315 ++++++++++++++++++ .../GroupByKey/GroupByKey/pkg/task/task.go | 25 ++ .../GroupByKey/GroupByKey/task-info.yaml | 35 ++ .../GroupByKey/task-remote-info.yaml | 2 + .../GroupByKey/GroupByKey/task.md | 54 +++ .../GroupByKey/GroupByKey/test/task_test.go | 57 ++++ .../GroupByKey/lesson-info.yaml | 21 ++ .../GroupByKey/lesson-remote-info.yaml | 3 + .../Map/ParDo OneToMany/task-remote-info.yaml | 2 +- .../Map/ParDo Struct/task-remote-info.yaml | 2 +- .../Map/ParDo/task-remote-info.yaml | 2 +- .../go/Core Transforms/section-info.yaml | 1 + .../Hello Beam/task-remote-info.yaml | 2 +- learning/katas/go/course-remote-info.yaml | 2 +- 16 files changed, 587 insertions(+), 5 deletions(-) create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/cmd/main.go create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.mod create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.sum create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/pkg/task/task.go create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-info.yaml create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-remote-info.yaml create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task.md create mode 100644 learning/katas/go/Core Transforms/GroupByKey/GroupByKey/test/task_test.go create mode 100644 learning/katas/go/Core Transforms/GroupByKey/lesson-info.yaml create mode 100644 learning/katas/go/Core Transforms/GroupByKey/lesson-remote-info.yaml diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/cmd/main.go b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/cmd/main.go new file mode 100644 index 000000000000..e2c077867af6 --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/cmd/main.go @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "context" + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/log" + "github.com/apache/beam/sdks/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/go/pkg/beam/x/debug" + "groupbykey/pkg/task" +) + +func main() { + ctx := context.Background() + + p, s := beam.NewPipelineWithRoot() + + input := beam.Create(s, "apple", "ball", "car", "bear", "cheetah", "ant") + + output := task.ApplyTransform(s, input) + + debug.Print(s, output) + + err := beamx.Run(ctx, p) + + if err != nil { + log.Exitf(context.Background(), "Failed to execute job: %v", err) + } +} diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.mod b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.mod new file mode 100644 index 000000000000..2cb9df14a34d --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.mod @@ -0,0 +1,26 @@ +// 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. + +module groupbykey + +go 1.14 + +require ( + github.com/apache/beam v2.20.0+incompatible + github.com/golang/protobuf v1.4.2 // indirect + github.com/google/go-cmp v0.4.1 + google.golang.org/api v0.24.0 // indirect + google.golang.org/grpc v1.29.1 // indirect +) diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.sum b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.sum new file mode 100644 index 000000000000..eb28a28159c5 --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/go.sum @@ -0,0 +1,315 @@ +cloud.google.com/go v0.26.0 h1:e0WKqKTd5BnrG8aKH3J3h+QvEIQtSUcf2n5UZ5ZgLtQ= +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.56.0 h1:WRz29PgAsVEyPSDHyk+0fpEkwEFyfhHn+JbksT6gIL4= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0 h1:xE3CPsOgttP4ACBePh79zTKALtXwn/Edhcr16R5hMWU= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0 h1:/May9ojXjRkPBNVrq+oWLqmWCkr4OU5uRY29bu0mRyQ= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0 h1:Lpy6hKgdcl7a3WGSfJIFmxmcdjSpP6OmBEfcOv1Y680= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0 h1:UDpwYIwla4jHGzZJaEJYx1tOejbgSoNqsAfHAUYe2r8= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/apache/beam v2.20.0+incompatible h1:YzP/+VyAnYdu4Wjh5EkBz3vUjVdE7vUPEZ6xijCJ2sk= +github.com/apache/beam v2.20.0+incompatible/go.mod h1:/8NX3Qi8vGstDLLaeaU7+lzVEu/ACaQhYjeefzQ0y1o= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1 h1:/exdXoGamhu5ONeUJH0deniYLWYvQwW66yvlfiiKTu0= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be h1:vEDujvNQGv4jgYKudGeI/+DAX4Jffq6hpD55MmoEvKs= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a h1:WXEvlFVvvGxCJLG6REjsT03iWnKLEWinaScsxF2Vm2o= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d h1:nc5K6ox/4lTFbMVSL9WRR81ixkcwXThoiF6yf+R9scA= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4 h1:kDtqNkeBrZb8B+atrj50B5XLHpzXXqcCdZPP/ApQ5NY= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0 h1:cG03eaksBzhfSIk7JRGctfp3lanklcOM/mTGvow7BbQ= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5 h1:tycE03LOZYQNhDpS27tcQdAzLCVMaj7QT2SXxebnpCM= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940 h1:MRHtG0U6SnaUb+s+LhNE1qt1FQ1wlhqr5E4usBKC0uA= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/pkg/task/task.go b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/pkg/task/task.go new file mode 100644 index 000000000000..1d9ce6d45c6b --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/pkg/task/task.go @@ -0,0 +1,25 @@ +// 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 task + +import "github.com/apache/beam/sdks/go/pkg/beam" + +func ApplyTransform(s beam.Scope, input beam.PCollection) beam.PCollection { + kv := beam.ParDo(s, func(element string) (string, string) { + return string(element[0]), element + }, input) + return beam.GroupByKey(s, kv) +} diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-info.yaml b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-info.yaml new file mode 100644 index 000000000000..b4aebb3cf48c --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-info.yaml @@ -0,0 +1,35 @@ +# +# 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. +# + +type: edu +files: +- name: test/task_test.go + visible: false +- name: cmd/main.go + visible: true +- name: go.mod + visible: false +- name: pkg/task/task.go + visible: true + placeholders: + - offset: 937 + length: 138 + placeholder_text: TODO() +- name: go.sum + visible: false diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-remote-info.yaml b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-remote-info.yaml new file mode 100644 index 000000000000..a766ed47f2b3 --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task-remote-info.yaml @@ -0,0 +1,2 @@ +id: 1325218 +update_date: Wed, 20 May 2020 06:15:19 UTC diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task.md b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task.md new file mode 100644 index 000000000000..66ce7c896d0a --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/task.md @@ -0,0 +1,54 @@ + + +# GroupByKey + +GroupByKey is a Beam transform for processing collections of key/value pairs. It’s a parallel +reduction operation, analogous to the Shuffle phase of a Map/Shuffle/Reduce-style algorithm. The +input to GroupByKey is a collection of key/value pairs that represents a multimap, where the +collection contains multiple pairs that have the same key, but different values. Given such a +collection, you use GroupByKey to collect all of the values associated with each unique key. + +**Kata:** Implement a + +beam.GroupByKey transform that groups words by its first letter. + +
    + Refer to + beam.GroupByKey to solve this problem. +
    + +
    + Refer to the Beam Programming Guide + + "GroupByKey" section for more information. +
    + +
    + To return as a KV, you can return two values from your DoFn. The first return value represents the Key, and + the second return value represents the Value. An example is shown below. + +``` +func doFn(element string) (string, string) { + key := string(element[0]) + value := element + return key, value +} +``` +
    diff --git a/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/test/task_test.go b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/test/task_test.go new file mode 100644 index 000000000000..7f9a8827a984 --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/GroupByKey/test/task_test.go @@ -0,0 +1,57 @@ +// 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 test + +import ( + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/testing/ptest" + "github.com/google/go-cmp/cmp" + "groupbykey/pkg/task" + "testing" +) + +func TestApplyTransform(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + tests := []struct { + input beam.PCollection + want map[string][]string + }{ + { + input: beam.Create(s, "apple", "ball", "car", "bear", "cheetah", "ant"), + want: map[string][]string{ + "a": {"apple", "ant"}, + "b": {"ball", "bear"}, + "c": {"car", "cheetah"}, + }, + }, + } + for _, tt := range tests { + got := task.ApplyTransform(s, tt.input) + beam.ParDo0(s, func(key string, values func(*string) bool) { + var got []string + var v string + for values(&v) { + got = append(got, v) + } + if !cmp.Equal(got, tt.want[key]) { + t.Errorf("ApplyTransform() key = %s, got %v , want %v", key, got, tt.want[key]) + } + }, got) + if err := ptest.Run(p); err != nil { + t.Error(err) + } + } +} diff --git a/learning/katas/go/Core Transforms/GroupByKey/lesson-info.yaml b/learning/katas/go/Core Transforms/GroupByKey/lesson-info.yaml new file mode 100644 index 000000000000..5de9eb69f819 --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/lesson-info.yaml @@ -0,0 +1,21 @@ +# +# 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. +# + +content: +- GroupByKey diff --git a/learning/katas/go/Core Transforms/GroupByKey/lesson-remote-info.yaml b/learning/katas/go/Core Transforms/GroupByKey/lesson-remote-info.yaml new file mode 100644 index 000000000000..6391a8192a9c --- /dev/null +++ b/learning/katas/go/Core Transforms/GroupByKey/lesson-remote-info.yaml @@ -0,0 +1,3 @@ +id: 358048 +update_date: Wed, 20 May 2020 06:15:15 UTC +unit: 342241 diff --git a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml index 2512ec8b7de9..8e0e17b2c09f 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289189 -update_date: Tue, 05 May 2020 02:01:57 UTC +update_date: Wed, 20 May 2020 06:15:28 UTC diff --git a/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml index 190348ed6ee7..5ca73c7400f6 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289190 -update_date: Tue, 05 May 2020 02:01:59 UTC +update_date: Wed, 20 May 2020 06:15:31 UTC diff --git a/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml index 72b8745db6fc..c5aaaae2a15f 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289188 -update_date: Tue, 05 May 2020 02:01:55 UTC +update_date: Wed, 20 May 2020 06:15:25 UTC diff --git a/learning/katas/go/Core Transforms/section-info.yaml b/learning/katas/go/Core Transforms/section-info.yaml index 23c69bef45f5..8ea49eb04ce4 100644 --- a/learning/katas/go/Core Transforms/section-info.yaml +++ b/learning/katas/go/Core Transforms/section-info.yaml @@ -19,3 +19,4 @@ content: - Map +- GroupByKey diff --git a/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml b/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml index bf68a02eca35..46551fb4c9fd 100644 --- a/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml +++ b/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1236831 -update_date: Thu, 16 Apr 2020 06:53:05 UTC +update_date: Wed, 20 May 2020 06:15:22 UTC diff --git a/learning/katas/go/course-remote-info.yaml b/learning/katas/go/course-remote-info.yaml index 4252852155fc..2651d8336da7 100644 --- a/learning/katas/go/course-remote-info.yaml +++ b/learning/katas/go/course-remote-info.yaml @@ -1,2 +1,2 @@ id: 70387 -update_date: Fri, 24 Apr 2020 07:39:05 UTC +update_date: Tue, 05 May 2020 02:02:55 UTC From 21da8903180e58244f817d956a080d5a8c5e66a8 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Tue, 19 May 2020 20:22:52 -0400 Subject: [PATCH 033/151] [BEAM-10048] Clean up release guide. - Remove redundancies between the release guide and build_release_candidate.sh. - Update website-related instructions for Hugo. --- .../main/scripts/build_release_candidate.sh | 18 +- .../content/en/contribute/release-guide.md | 212 +++--------------- 2 files changed, 36 insertions(+), 194 deletions(-) diff --git a/release/src/main/scripts/build_release_candidate.sh b/release/src/main/scripts/build_release_candidate.sh index 36548d530f68..be3d741bb06c 100755 --- a/release/src/main/scripts/build_release_candidate.sh +++ b/release/src/main/scripts/build_release_candidate.sh @@ -113,7 +113,7 @@ if [[ $confirmation = "y" ]]; then echo "-------------Staging Java Artifacts into Maven---------------" gpg --local-user ${SIGNING_KEY} --output /dev/null --sign ~/.bashrc ./gradlew publish -Psigning.gnupg.keyName=${SIGNING_KEY} -PisRelease --no-daemon - echo "Please review all artifacts in staging URL. e.g. https://repository.apache.org/content/repositories/orgapachebeam-NNNN/" + echo "You need to close the staging repository manually on Apache Nexus. See the release guide for instructions." rm -rf ~/${LOCAL_CLONE_DIR} fi @@ -205,7 +205,7 @@ if [[ $confirmation = "y" ]]; then rm -rf ~/${PYTHON_ARTIFACTS_DIR} fi -echo "[Current Step]: Stage SDK docker images" +echo "[Current Step]: Stage docker images" echo "Do you want to proceed? [y|N]" read confirmation if [[ $confirmation = "y" ]]; then @@ -330,18 +330,6 @@ if [[ $confirmation = "y" ]]; then fi echo "===========Please Review All Items in the Checklist==========" -echo "1. Maven artifacts deployed to https://repository.apache.org/content/repositories/" +echo "1. You need to close the staging repository manually on Apache Nexus. See the release guide for instructions." echo "2. Source distribution deployed to https://dist.apache.org/repos/dist/dev/beam/${RELEASE}" echo "3. Website pull request published the Java API reference manual the Python API reference manual." - -echo "==============Things Needed To Be Done Manually==============" -echo "1.Make sure a pull request is created to update the javadoc and pydoc to the beam-site: " -echo " - cd ~/${LOCAL_WEBSITE_UPDATE_DIR}/${LOCAL_WEBSITE_REPO}/${WEBSITE_ROOT_DIR}" -echo " - git checkout updates_release_${RELEASE}" -echo " - Check if both javadoc/ and pydoc/ exist." -echo " - commit your changes" -echo "2.Create a pull request to update the release in the beam/website:" -echo " - An example pull request:https://github.com/apache/beam/pull/9341" -echo " - You can find the release note in JIRA: https://issues.apache.org/jira/projects/BEAM?selectedItem=com.atlassian.jira.jira-projects-plugin%3Arelease-page&status=unreleased" -echo "3.You need to build Python Wheels." -echo "4.Start the review-and-vote thread on the dev@ mailing list." diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 95c95c2272bb..3da9fbcc3973 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -171,7 +171,6 @@ Configure access to the [Apache Nexus repository](https://repository.apache.org/ -__NOTE__: make sure the XML you end up with matches the structure above. #### Submit your GPG public key into MIT PGP Public Key Server In order to make yourself have right permission to stage java artifacts in Apache Nexus staging repository, @@ -583,189 +582,44 @@ For this step, we recommend you using automation script to create a RC, but you 1. Stage source release into dist.apache.org dev [repo](https://dist.apache.org/repos/dist/dev/beam/). 1. Stage,sign and hash python binaries into dist.apache.ord dev repo python dir 1. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). - 1. Create a PR to update beam and beam-site, changes includes: + 1. Create a PR to update beam-site, changes includes: * Copy python doc into beam-site * Copy java doc into beam-site - * Update release version into [_config.yml](https://github.com/apache/beam/blob/master/website/_config.yml). #### Tasks you need to do manually - 1. Add new release into `website/src/get-started/downloads.md`. - 1. Update last release download links in `website/src/get-started/downloads.md`. - 1. Update `website/src/.htaccess` to redirect to the new version. - 1. Build and stage python wheels. + 1. Verify the script worked. + 1. Verify that the source and Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). + 1. Verify Docker images are published. How to find images: + 1. Visit [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image) + 2. Visit each repository and navigate to *tags* tab. + 3. Verify images are pushed with tags: ${RELEASE}_rc{RC_NUM} + 1. Verify that third party licenses are included in Docker containers by logging in to the images. + - For Python SDK images, there should be around 80 ~ 100 dependencies. + Please note that dependencies for the SDKs with different Python versions vary. + Need to verify all Python images by replacing `${ver}` with each supported Python version `X.Y`. + ``` + docker run -it --entrypoint=/bin/bash apache/beam_python${ver}_sdk:${RELEASE}_rc{RC_NUM} + ls -al /opt/apache/beam/third_party_licenses/ | wc -l + ``` + - For Java SDK images, there should be around 1400 dependencies. + ``` + docker run -it --entrypoint=/bin/bash apache/beam_java_sdk:${RELEASE}_rc{RC_NUM} + ls -al /opt/apache/beam/third_party_licenses/ | wc -l + ``` 1. Publish staging artifacts - 1. Go to the staging repo to close the staging repository on [Apache Nexus](https://repository.apache.org/#stagingRepositories). + 1. Go to the staging repo to close the staging repository on [Apache Nexus](https://repository.apache.org/#stagingRepositories). 1. When prompted for a description, enter “Apache Beam, version X, release candidate Y”. - - -### (Alternative) Run all steps manually - -#### Build and stage Java artifacts with Gradle - -Set up a few environment variables to simplify the commands that follow. These identify the release candidate being built, and the branch where you will stage files. Start with `RC_NUM` equal to `1` and increment it for each candidate. - - RC_NUM=1 - -Make sure your git config will maintain your account: - - git config credential.helper store - -Use Gradle release plugin to build the release artifacts, and push code and -release tag to the origin repository (this would be the Apache Beam repo): - - ./gradlew release -Prelease.newVersion=${RELEASE}-SNAPSHOT \ - -Prelease.releaseVersion=${RELEASE}-RC${RC_NUM} \ - -Prelease.useAutomaticVersion=true --info --no-daemon - -Use Gradle publish plugin to stage these artifacts on the Apache Nexus repository, as follows: - - ./gradlew publish -PisRelease --no-parallel --no-daemon - -Review all staged artifacts. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.beam%22) in which artifact name mirrors directory structure, e.g., `beam-sdks-java-io-kafka`. Carefully review any new artifacts. - -Close the staging repository on Apache Nexus. When prompted for a description, enter “Apache Beam, version X, release candidate Y”. - -#### Stage source release on dist.apache.org - -Attention: Only committer has permissions to perform following steps. - -Copy the source release to the dev repository of `dist.apache.org`. - -1. If you have not already, check out the Beam section of the `dev` repository on `dist.apache.org` via Subversion. In a fresh directory: - - svn co https://dist.apache.org/repos/dist/dev/beam - -1. Make a directory for the new release: - - mkdir beam/${RELEASE} - cd beam/${RELEASE} - -1. Download source zip from GitHub: - - wget https://github.com/apache/beam/archive/release-${RELEASE}.zip \ - -O apache-beam-${RELEASE}-source-release.zip - -1. Create hashes and sign the source distribution: - - gpg --armor --detach-sig apache-beam-${RELEASE}-source-release.zip - sha512sum apache-beam-${RELEASE}-source-release.zip > apache-beam-${RELEASE}-source-release.zip.sha512 - -1. Add and commit all the files. - - svn add beam/${RELEASE} - svn commit - -1. Verify that files are [present](https://dist.apache.org/repos/dist/dev/beam). - -#### Stage python binaries on dist.apache.org - -Build python binaries in release branch in sdks/python dir. - - pip install -r build-requirements.txt - python setup.py sdist --format=zip - cd dist - cp apache-beam-${RELEASE}.zip staging/apache-beam-${RELEASE}-python.zip - cd staging - -Create hashes and sign the binaries - - gpg --armor --detach-sig apache-beam-${RELEASE}-python.zip - sha512sum apache-beam-${RELEASE}-python.zip > apache-beam-${RELEASE}-python.zip.sha512 - -Staging binaries - - svn co https://dist.apache.org/repos/dist/dev/beam - cd beam/${RELEASE} - svn add * - svn commit - -Verify that files are [present](https://dist.apache.org/repos/dist/dev/beam). - -#### Stage SDK images on hub.docker.com -* Build Python images and push to DockerHub. - -``` -./gradlew :sdks:python:container:buildAll -Pdocker-pull-licenses -Pdocker-tag=${RELEASE}_rc{RC_NUM} -``` - -Verify that third party licenses are included by logging in to the images. For Python SDK images, there should be around 80 ~ 100 dependencies. -Please note that dependencies for the SDKs with different Python versions vary. -Need to verify all Python images by replacing `${ver}` in the following command to `python2.7, python3.5, python3.6, python3.7`. - -``` -docker run -it --entrypoint=/bin/bash apache/beam_${ver}_sdk:${RELEASE}_rc{RC_NUM} -ls -al /opt/apache/beam/third_party_licenses/ | wc -l -``` - -After verifying the third party licenses are included correctly, push the images to DockerHub. -``` -PYTHON_VER=("python2.7" "python3.5" "python3.6" "python3.7") -for ver in "${PYTHON_VER[@]}"; do - docker push apache/beam_${ver}_sdk:${RELEASE}_rc{RC_NUM} & -done -``` - -* Build Java images and push to DockerHub. - -``` -./gradlew :sdks:java:container:docker -Pdocker-pull-licenses -Pdocker-tag=${RELEASE}_rc{RC_NUM} -``` - -Verify that third party licenses are included by logging in to the images. For Java SDK images, there should be around 1400 dependencies. -``` -docker run -it --entrypoint=/bin/bash apache/beam_java_sdk:${RELEASE}_rc{RC_NUM} -ls -al /opt/apache/beam/third_party_licenses/ | wc -l -``` - -After verifying the third party licenses are included correctly, push the images to DockerHub. -``` -docker push apache/beam_java_sdk:${RELEASE}_rc{RC_NUM} -``` - -* Build Flink job server images and push to DockerHub. - -``` -FLINK_VER=("1.8" "1.9" "1.10") -for ver in "${FLINK_VER[@]}"; do - ./gradlew ":runners:flink:${ver}:job-server-container:dockerPush" -Pdocker-tag="${RELEASE}_rc${RC_NUM}" -done -``` - -* Build Spark job server image and push to DockerHub. - -``` -./gradlew ":runners:spark:job-server:container:dockerPush" -Pdocker-tag="${RELEASE}_rc${RC_NUM}" -``` - -Clean up images from local - -``` -for ver in "${PYTHON_VER[@]}"; do - docker rmi -f apache/beam_${ver}_sdk:${RELEASE}_rc{RC_NUM} -done -docker rmi -f apache/beam_java_sdk:${RELEASE}_rc{RC_NUM} -for ver in "${FLINK_VER[@]}"; do - docker rmi -f "apache/beam_flink${ver}_job_server:${RELEASE}_rc${RC_NUM}" -done -docker rmi -f "apache/beam_spark_job_server:${RELEASE}_rc${RC_NUM}" -``` - -How to find images: -1. Visit [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image) -2. Visit each repository and navigate to *tags* tab. -3. Verify images are pushed with tags: ${RELEASE}_rc{RC_NUM} - -### Build and stage python wheels - -There is a wrapper repo [beam-wheels](https://github.com/apache/beam-wheels) to help build python wheels. - -If you are interested in how it works, please refer to the [structure section](https://github.com/apache/beam-wheels#structure). - -Please follow the [user guide](https://github.com/apache/beam-wheels#user-guide) to build python wheels. - -Once all python wheels have been staged [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/), -please run [./sign_hash_python_wheels.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/sign_hash_python_wheels.sh) to sign and hash python wheels. - + 1. Review all staged artifacts on https://repository.apache.org/content/repositories/orgapachebeam-NNNN/. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.beam%22) in which artifact name mirrors directory structure, e.g., `beam-sdks-java-io-kafka`. Carefully review any new artifacts. + 1. Build and stage python wheels. + - There is a wrapper repo [beam-wheels](https://github.com/apache/beam-wheels) to help build python wheels. + - If you are interested in how it works, please refer to the [structure section](https://github.com/apache/beam-wheels#structure). + - Please follow the [user guide](https://github.com/apache/beam-wheels#user-guide) to build python wheels. + - Once all python wheels have been staged [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/), + please run [./sign_hash_python_wheels.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/sign_hash_python_wheels.sh) to sign and hash python wheels. + 1. Update Beam website ([example](https://github.com/apache/beam/pull/11727)) + 1. Update release version in `website/www/site/config.toml`. + 1. Add new release in `website/www/site/content/en/get-started/downloads.md`. + 1. Update `website/www/site/static/.htaccess` to redirect to the new version. ********** @@ -842,7 +696,7 @@ This pull request is against the `apache/beam` repo, on the `master` branch. ### Blog post -Write a blog post similar to https://beam.apache.org/blog/2019/08/22/beam-2.15.0.html +Write a blog post similar to [beam-2.20.0.md](https://github.com/apache/beam/blob/master/website/www/site/content/en/blog/beam-2.20.0.md). - Update `CHANGES.md` by adding a new section for the next release. - Copy the changes for the current release from `CHANGES.md` to the blog post and edit as necessary. From 1404c6a1ab3ca7e9ff3267c7bd28e18472c33e2a Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 20 May 2020 17:10:21 -0400 Subject: [PATCH 034/151] Remove checklist from build_release_candidate.sh. It is included in the release guide. --- release/src/main/scripts/build_release_candidate.sh | 5 ----- 1 file changed, 5 deletions(-) diff --git a/release/src/main/scripts/build_release_candidate.sh b/release/src/main/scripts/build_release_candidate.sh index be3d741bb06c..df5470bc2dec 100755 --- a/release/src/main/scripts/build_release_candidate.sh +++ b/release/src/main/scripts/build_release_candidate.sh @@ -328,8 +328,3 @@ if [[ $confirmation = "y" ]]; then rm -rf ~/${LOCAL_WEBSITE_UPDATE_DIR}/${LOCAL_JAVA_DOC} rm -rf ~/${LOCAL_WEBSITE_UPDATE_DIR}/${LOCAL_PYTHON_DOC} fi - -echo "===========Please Review All Items in the Checklist==========" -echo "1. You need to close the staging repository manually on Apache Nexus. See the release guide for instructions." -echo "2. Source distribution deployed to https://dist.apache.org/repos/dist/dev/beam/${RELEASE}" -echo "3. Website pull request published the Java API reference manual the Python API reference manual." From 74a53409a39de1549f9fb33f40b2266ef49f4fdc Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Wed, 20 May 2020 14:27:07 -0700 Subject: [PATCH 035/151] Clarifies text. --- .../apache_beam/runners/dataflow/internal/apiclient.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index d0e7911264ed..ee1e4720548d 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -316,10 +316,10 @@ def __init__( # Currently, Dataflow uses Docker container images to uniquely # identify execution environments. Hence Dataflow executes all - # transforms that specify the same Docker container image in the - # same container. Dependencies of all environments that specify a - # given container image will be staged in the container for that - # particular container image. + # transforms that specify the the same Docker container image in a + # single container instance. Dependencies of all environments that + # specify a given container image will be staged in the container + # instance for that particular container image. # TODO(BEAM-9455): loosen this restriction to support multiple # environments with the same container image when Dataflow supports # environment specific artifact provisioning. From 9818fa823a4ef35840a281503b3402723b5c82fe Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 20 May 2020 14:59:41 -0700 Subject: [PATCH 036/151] [BEAM-9699] Add test verifying we can use ZetaSQL in Python SqlTransform (#11575) * Add zetasql to SQL expansion service * Add support for ZetaSQL in Python SqlTransform plumbs through queryPlannerClassName param * Move dialect -> planner class resolution to Java Requires moving ExternalTransform classes to extensions/sql/expansion-service so we can reference ZetaSQL * add expansion-service package-info (checkstyle) --- .../sql/expansion-service/build.gradle | 1 + .../ExternalSqlTransformRegistrar.java | 82 +++++++++++++++++++ .../sql/expansion/package-info.java | 20 +++++ .../beam/sdk/extensions/sql/SqlTransform.java | 39 +-------- sdks/python/apache_beam/transforms/sql.py | 8 +- .../python/apache_beam/transforms/sql_test.py | 10 +++ 6 files changed, 119 insertions(+), 41 deletions(-) create mode 100644 sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java create mode 100644 sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/package-info.java diff --git a/sdks/java/extensions/sql/expansion-service/build.gradle b/sdks/java/extensions/sql/expansion-service/build.gradle index 19a32709522b..998d3cae5f2d 100644 --- a/sdks/java/extensions/sql/expansion-service/build.gradle +++ b/sdks/java/extensions/sql/expansion-service/build.gradle @@ -30,6 +30,7 @@ ext.summary = """Contains code to run a SQL Expansion Service.""" dependencies { compile project(path: ":sdks:java:extensions:sql") + compile project(path: ":sdks:java:extensions:sql:zetasql") compile project(path: ":sdks:java:expansion-service") } diff --git a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java new file mode 100644 index 000000000000..cec0a867bc0d --- /dev/null +++ b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.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.beam.sdk.extensions.sql.expansion; + +import com.google.auto.service.AutoService; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; +import org.apache.beam.sdk.extensions.sql.SqlTransform; +import org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner; +import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner; +import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner; +import org.apache.beam.sdk.transforms.ExternalTransformBuilder; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; + +@AutoService(ExternalTransformRegistrar.class) +public class ExternalSqlTransformRegistrar implements ExternalTransformRegistrar { + private static final String URN = "beam:external:java:sql:v1"; + private static final ImmutableMap> DIALECTS = + ImmutableMap.>builder() + .put("zetasql", ZetaSQLQueryPlanner.class) + .put("calcite", CalciteQueryPlanner.class) + .build(); + + @Override + public Map> knownBuilders() { + return org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap.of( + URN, Builder.class); + } + + public static class Configuration { + String query; + String dialect; + + public void setQuery(String query) { + this.query = query; + } + + public void setDialect(@Nullable String dialect) { + this.dialect = dialect; + } + } + + private static class Builder + implements ExternalTransformBuilder> { + @Override + public PTransform> buildExternal(Configuration configuration) { + SqlTransform transform = SqlTransform.query(configuration.query); + if (configuration.dialect != null) { + Class queryPlanner = + DIALECTS.get(configuration.dialect.toLowerCase()); + if (queryPlanner == null) { + throw new IllegalArgumentException( + String.format( + "Received unknown SQL Dialect '%s'. Known dialects: %s", + configuration.dialect, DIALECTS.keySet())); + } + transform = transform.withQueryPlannerClass(queryPlanner); + } + return transform; + } + } +} diff --git a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/package-info.java b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/package-info.java new file mode 100644 index 000000000000..1dcf3fca9e16 --- /dev/null +++ b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** External Transform Registration for Beam SQL. */ +package org.apache.beam.sdk.extensions.sql.expansion; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 496dd6d37f46..12ce337f9685 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.extensions.sql; -import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.HashMap; @@ -25,7 +24,6 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.BeamSqlEnvBuilder; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; @@ -37,7 +35,6 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.ExternalTransformBuilder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; @@ -273,8 +270,7 @@ static Builder builder() { } @AutoValue.Builder - abstract static class Builder - implements ExternalTransformBuilder> { + abstract static class Builder { abstract Builder setQueryString(String queryString); abstract Builder setQueryParameters(QueryParameters queryParameters); @@ -292,19 +288,6 @@ abstract static class Builder abstract Builder setQueryPlannerClassName(@Nullable String queryPlannerClassName); abstract SqlTransform build(); - - @Override - public PTransform> buildExternal( - External.Configuration configuration) { - return builder() - .setQueryString(configuration.query) - .setQueryParameters(QueryParameters.ofNone()) - .setUdafDefinitions(Collections.emptyList()) - .setUdfDefinitions(Collections.emptyList()) - .setTableProviderMap(Collections.emptyMap()) - .setAutoUdfUdafLoad(false) - .build(); - } } @AutoValue @@ -330,24 +313,4 @@ static UdafDefinition of(String udafName, Combine.CombineFn combineFn) { return new AutoValue_SqlTransform_UdafDefinition(udafName, combineFn); } } - - @AutoService(ExternalTransformRegistrar.class) - public static class External implements ExternalTransformRegistrar { - - private static final String URN = "beam:external:java:sql:v1"; - - @Override - public Map> knownBuilders() { - return org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap.of( - URN, AutoValue_SqlTransform.Builder.class); - } - - public static class Configuration { - String query; - - public void setQuery(String query) { - this.query = query; - } - } - } } diff --git a/sdks/python/apache_beam/transforms/sql.py b/sdks/python/apache_beam/transforms/sql.py index 86423761b8b6..ce409d21fd26 100644 --- a/sdks/python/apache_beam/transforms/sql.py +++ b/sdks/python/apache_beam/transforms/sql.py @@ -32,7 +32,8 @@ __all__ = ['SqlTransform'] SqlTransformSchema = typing.NamedTuple( - 'SqlTransformSchema', [('query', unicode)]) + 'SqlTransformSchema', [('query', unicode), + ('dialect', typing.Optional[unicode])]) class SqlTransform(ExternalTransform): @@ -66,9 +67,10 @@ class SqlTransform(ExternalTransform): """ URN = 'beam:external:java:sql:v1' - def __init__(self, query): + def __init__(self, query, dialect=None): super(SqlTransform, self).__init__( self.URN, - NamedTupleBasedPayloadBuilder(SqlTransformSchema(query=query)), + NamedTupleBasedPayloadBuilder( + SqlTransformSchema(query=query, dialect=dialect)), BeamJarExpansionService( ':sdks:java:extensions:sql:expansion-service:shadowJar')) diff --git a/sdks/python/apache_beam/transforms/sql_test.py b/sdks/python/apache_beam/transforms/sql_test.py index b8097145def0..d2271bbfd923 100644 --- a/sdks/python/apache_beam/transforms/sql_test.py +++ b/sdks/python/apache_beam/transforms/sql_test.py @@ -138,6 +138,16 @@ def test_tagged_join(self): ON simple.`int` = enrich.`int`""")) assert_that(out, equal_to([(1, "a"), (26, "z"), (1, "a")])) + def test_zetasql_generate_data(self): + with TestPipeline() as p: + out = p | SqlTransform( + """SELECT + CAST(1 AS INT64) AS `int`, + CAST('foo' AS STRING) AS `str`, + CAST(3.14 AS FLOAT64) AS `flt`""", + dialect="zetasql") + assert_that(out, equal_to([(1, "foo", 3.14)])) + if __name__ == "__main__": logging.getLogger().setLevel(logging.INFO) From 758d821ac64edebf24bc1bba1f7bfa34fd03a2da Mon Sep 17 00:00:00 2001 From: omarismail94 <44980219+omarismail94@users.noreply.github.com> Date: Wed, 20 May 2020 18:01:15 -0400 Subject: [PATCH 037/151] [BEAM-10037] BeamSqlExample.java fails to build (#11754) * [BEAM-10037] BeamSqlExample.java fails to build when running ./gradlew command * [BEAM-10037] Added BeamSQLExample and PojoExample to sqlPreCommit(), and fixed coder in BeamSqlExample * [BEAM-10037] Reverted first query to c1>1 as opposed to c3. I added c3 for testing and forgot to change it back :) Co-authored-by: Omar Ismail --- build.gradle | 2 + .../sql/example/BeamSqlExample.java | 56 ++++++++++--------- 2 files changed, 33 insertions(+), 25 deletions(-) diff --git a/build.gradle b/build.gradle index ca4b54ea2710..c26a02067da3 100644 --- a/build.gradle +++ b/build.gradle @@ -152,6 +152,8 @@ task javaPreCommit() { } task sqlPreCommit() { + dependsOn ":sdks:java:extensions:sql:runBasicExample" + dependsOn ":sdks:java:extensions:sql:runPojoExample" dependsOn ":sdks:java:extensions:sql:build" dependsOn ":sdks:java:extensions:sql:buildDependents" } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java index 8496a712fef0..9742a3384b33 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java @@ -45,6 +45,7 @@ * Beam documentation on how to run pipelines. */ class BeamSqlExample { + public static void main(String[] args) { PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); Pipeline p = Pipeline.create(options); @@ -66,19 +67,21 @@ public static void main(String[] args) { inputTable.apply(SqlTransform.query("select c1, c2, c3 from PCOLLECTION where c1 > 1")); // print the output record of case 1; - outputStream.apply( - "log_result", - MapElements.via( - new SimpleFunction() { - @Override - public Row apply(Row input) { - // expect output: - // PCOLLECTION: [3, row, 3.0] - // PCOLLECTION: [2, row, 2.0] - System.out.println("PCOLLECTION: " + input.getValues()); - return input; - } - })); + outputStream + .apply( + "log_result", + MapElements.via( + new SimpleFunction() { + @Override + public Row apply(Row input) { + // expect output: + // PCOLLECTION: [3, row, 3.0] + // PCOLLECTION: [2, row, 2.0] + System.out.println("PCOLLECTION: " + input.getValues()); + return input; + } + })) + .setRowSchema(type); // Case 2. run the query with SqlTransform.query over result PCollection of case 1. PCollection outputStream2 = @@ -86,18 +89,21 @@ public Row apply(Row input) { .apply(SqlTransform.query("select c2, sum(c3) from CASE1_RESULT group by c2")); // print the output record of case 2; - outputStream2.apply( - "log_result", - MapElements.via( - new SimpleFunction() { - @Override - public Row apply(Row input) { - // expect output: - // CASE1_RESULT: [row, 5.0] - System.out.println("CASE1_RESULT: " + input.getValues()); - return input; - } - })); + outputStream2 + .apply( + "log_result", + MapElements.via( + new SimpleFunction() { + @Override + public Row apply(Row input) { + // expect output: + // CASE1_RESULT: [row, 5.0] + System.out.println("CASE1_RESULT: " + input.getValues()); + return input; + } + })) + .setRowSchema( + Schema.builder().addStringField("stringField").addDoubleField("doubleField").build()); p.run().waitUntilFinish(); } From e1b148bf1f2e735ee082c06e1097fe72bb152365 Mon Sep 17 00:00:00 2001 From: Boyuan Zhang Date: Tue, 12 May 2020 20:24:56 -0700 Subject: [PATCH 038/151] Implement GrowableOffsetRangeTracker --- .../GrowableOffsetRangeTracker.java | 143 ++++++++++ .../splittabledofn/OffsetRangeTracker.java | 58 +++- .../GrowableOffsetRangeTrackerTest.java | 255 ++++++++++++++++++ .../OffsetRangeTrackerTest.java | 43 +++ 4 files changed, 485 insertions(+), 14 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java new file mode 100644 index 000000000000..a74d72b72b3d --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java @@ -0,0 +1,143 @@ +/* + * 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.beam.sdk.transforms.splittabledofn; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.math.BigDecimal; +import java.math.MathContext; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers; + +/** + * An {@link OffsetRangeTracker} for tracking a growable offset range. {@code Long.MAX_VALUE} is + * used as the end of the range to indicate infinity. + * + *

    An offset range is considered growable when the end offset could grow (or change) during + * execution time (e.g., Kafka topic partition offset, appended file, ...). + * + *

    The growable range is marked as done by claiming {@code Long.MAX_VALUE}. + */ +@Experimental(Kind.SPLITTABLE_DO_FN) +public class GrowableOffsetRangeTracker extends OffsetRangeTracker { + /** + * Provides the estimated end offset of the range. + * + *

    {@link #estimate} is called to give the end offset when {@link #trySplit} or {@link + * #getProgress} is invoked. The end offset is exclusive for the range. The estimated end is not + * required to monotonically increase as it will only be taken into consideration when the + * estimated end offset is larger than the current position. Returning {@code Long.MAX_VALUE} as + * the estimate implies the largest possible position for the range is {@code Long.MAX_VALUE - 1}. + * Return {@code Long.MIN_VALUE} if an estimate can not be provided. + * + *

    Providing a good estimate is important for an accurate progress signal and will impact + * splitting decisions by the runner. + * + *

    If {@link #estimate} is expensive to compute, consider wrapping the implementation with + * {@link Suppliers#memoizeWithExpiration} or equivalent as an optimization. + * + *

    TODO(BEAM-10032): Also consider using {@link RangeEndEstimator} when the range is not ended + * with {@code Long.MAX_VALUE}. + */ + @FunctionalInterface + public interface RangeEndEstimator { + long estimate(); + } + + private final RangeEndEstimator rangeEndEstimator; + + public GrowableOffsetRangeTracker(long start, RangeEndEstimator rangeEndEstimator) { + super(new OffsetRange(start, Long.MAX_VALUE)); + this.rangeEndEstimator = checkNotNull(rangeEndEstimator); + } + + @Override + public SplitResult trySplit(double fractionOfRemainder) { + // If current tracking range is no longer growable, split it as a normal range. + if (range.getTo() != Long.MAX_VALUE || range.getTo() == range.getFrom()) { + return super.trySplit(fractionOfRemainder); + } + // If current range has been done, there is no more space to split. + if (lastAttemptedOffset != null && lastAttemptedOffset == Long.MAX_VALUE) { + return null; + } + BigDecimal cur = + (lastAttemptedOffset == null) + ? BigDecimal.valueOf(range.getFrom()).subtract(BigDecimal.ONE, MathContext.DECIMAL128) + : BigDecimal.valueOf(lastAttemptedOffset); + + // Fetch the estimated end offset. If the estimated end is smaller than the next offset, use + // the next offset as end. + BigDecimal estimateRangeEnd = + BigDecimal.valueOf(rangeEndEstimator.estimate()) + .max(cur.add(BigDecimal.ONE, MathContext.DECIMAL128)); + + // Convert to BigDecimal in computation to prevent overflow, which may result in loss of + // precision. + // split = cur + max(1, (estimateRangeEnd - cur) * fractionOfRemainder) + BigDecimal splitPos = + cur.add( + estimateRangeEnd + .subtract(cur, MathContext.DECIMAL128) + .multiply(BigDecimal.valueOf(fractionOfRemainder), MathContext.DECIMAL128) + .max(BigDecimal.ONE), + MathContext.DECIMAL128); + long split = splitPos.longValue(); + if (split > estimateRangeEnd.longValue()) { + return null; + } + OffsetRange res = new OffsetRange(split, range.getTo()); + this.range = new OffsetRange(range.getFrom(), split); + return SplitResult.of(range, res); + } + + @Override + public Progress getProgress() { + // If current tracking range is no longer growable, get progress as a normal range. + if (range.getTo() != Long.MAX_VALUE || range.getTo() == range.getFrom()) { + return super.getProgress(); + } + + // Convert to BigDecimal in computation to prevent overflow, which may result in lost of + // precision. + BigDecimal estimateRangeEnd = BigDecimal.valueOf(rangeEndEstimator.estimate()); + + if (lastAttemptedOffset == null) { + return Progress.from( + 0, + estimateRangeEnd + .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128) + .max(BigDecimal.ZERO) + .doubleValue()); + } + + BigDecimal workRemaining = + estimateRangeEnd + .subtract(BigDecimal.valueOf(lastAttemptedOffset), MathContext.DECIMAL128) + .max(BigDecimal.ZERO); + BigDecimal totalWork = + estimateRangeEnd + .max(BigDecimal.valueOf(lastAttemptedOffset)) + .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128); + return Progress.from( + totalWork.subtract(workRemaining, MathContext.DECIMAL128).doubleValue(), + workRemaining.doubleValue()); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java index 2743a1dc56c4..9c7116f5e8c1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java @@ -21,6 +21,8 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import java.math.BigDecimal; +import java.math.MathContext; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -31,13 +33,16 @@ /** * A {@link RestrictionTracker} for claiming offsets in an {@link OffsetRange} in a monotonically * increasing fashion. + * + *

    The smallest offset is {@code Long.MIN_VALUE} and the largest offset is {@code Long.MAX_VALUE + * - 1}. */ @Experimental(Kind.SPLITTABLE_DO_FN) public class OffsetRangeTracker extends RestrictionTracker implements HasProgress { - private OffsetRange range; - @Nullable private Long lastClaimedOffset = null; - @Nullable private Long lastAttemptedOffset = null; + protected OffsetRange range; + @Nullable protected Long lastClaimedOffset = null; + @Nullable protected Long lastAttemptedOffset = null; public OffsetRangeTracker(OffsetRange range) { this.range = checkNotNull(range); @@ -50,16 +55,27 @@ public OffsetRange currentRestriction() { @Override public SplitResult trySplit(double fractionOfRemainder) { - long cur = (lastAttemptedOffset == null) ? range.getFrom() - 1 : lastAttemptedOffset; - long splitPos = - cur - + Math.max( - 1L, (Double.valueOf((range.getTo() - cur) * fractionOfRemainder)).longValue()); - if (splitPos >= range.getTo()) { + // Convert to BigDecimal in computation to prevent overflow, which may result in loss of + // precision. + BigDecimal cur = + (lastAttemptedOffset == null) + ? BigDecimal.valueOf(range.getFrom()).subtract(BigDecimal.ONE, MathContext.DECIMAL128) + : BigDecimal.valueOf(lastAttemptedOffset); + // split = cur + max(1, (range.getTo() - cur) * fractionOfRemainder) + BigDecimal splitPos = + cur.add( + BigDecimal.valueOf(range.getTo()) + .subtract(cur, MathContext.DECIMAL128) + .multiply(BigDecimal.valueOf(fractionOfRemainder), MathContext.DECIMAL128) + .max(BigDecimal.ONE), + MathContext.DECIMAL128); + + long split = splitPos.longValue(); + if (split >= range.getTo()) { return null; } - OffsetRange res = new OffsetRange(splitPos, range.getTo()); - this.range = new OffsetRange(range.getFrom(), splitPos); + OffsetRange res = new OffsetRange(split, range.getTo()); + this.range = new OffsetRange(range.getFrom(), split); return SplitResult.of(range, res); } @@ -120,13 +136,27 @@ public String toString() { public Progress getProgress() { // If we have never attempted an offset, we return the length of the entire range as work // remaining. + // Convert to BigDecimal in computation to prevent overflow, which may result in loss of + // precision. if (lastAttemptedOffset == null) { - return Progress.from(0, range.getTo() - range.getFrom()); + return Progress.from( + 0, + BigDecimal.valueOf(range.getTo()) + .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128) + .doubleValue()); } // Compute the amount of work remaining from where we are to where we are attempting to get to // with a minimum of zero in case we have claimed beyond the end of the range. - long workRemaining = Math.max(range.getTo() - lastAttemptedOffset, 0); - return Progress.from(range.getTo() - range.getFrom() - workRemaining, workRemaining); + BigDecimal workRemaining = + BigDecimal.valueOf(range.getTo()) + .subtract(BigDecimal.valueOf(lastAttemptedOffset), MathContext.DECIMAL128) + .max(BigDecimal.ZERO); + BigDecimal totalWork = + BigDecimal.valueOf(range.getTo()) + .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128); + return Progress.from( + totalWork.subtract(workRemaining, MathContext.DECIMAL128).doubleValue(), + workRemaining.doubleValue()); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java new file mode 100644 index 000000000000..17050157d4cc --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java @@ -0,0 +1,255 @@ +/* + * 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.beam.sdk.transforms.splittabledofn; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.math.BigDecimal; +import java.math.MathContext; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.Progress; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link GrowableOffsetRangeTracker}. */ +@RunWith(JUnit4.class) +public class GrowableOffsetRangeTrackerTest { + private static class SimpleEstimator implements GrowableOffsetRangeTracker.RangeEndEstimator { + private long estimateRangeEnd = 0; + + @Override + public long estimate() { + return estimateRangeEnd; + } + + public void setEstimateRangeEnd(long offset) { + estimateRangeEnd = offset; + } + } + + @Rule public final ExpectedException expected = ExpectedException.none(); + + @Test + public void testIllegalInitialization() throws Exception { + expected.expect(NullPointerException.class); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, null); + } + + @Test + public void testTryClaim() throws Exception { + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, new SimpleEstimator()); + assertTrue(tracker.tryClaim(10L)); + assertTrue(tracker.tryClaim(100L)); + assertFalse(tracker.tryClaim(Long.MAX_VALUE)); + tracker.checkDone(); + } + + @Test + public void testCheckpointBeforeStart() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + simpleEstimator.setEstimateRangeEnd(10); + SplitResult res = tracker.trySplit(0); + tracker.checkDone(); + assertEquals(new OffsetRange(0, 0), res.getPrimary()); + assertEquals(new OffsetRange(0, 0), tracker.currentRestriction()); + assertEquals(new OffsetRange(0, Long.MAX_VALUE), res.getResidual()); + } + + @Test + public void testCheckpointJustStarted() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + assertTrue(tracker.tryClaim(5L)); + simpleEstimator.setEstimateRangeEnd(0L); + SplitResult res = tracker.trySplit(0); + tracker.checkDone(); + assertEquals(new OffsetRange(0, 6), res.getPrimary()); + assertEquals(new OffsetRange(0, 6), tracker.currentRestriction()); + assertEquals(new OffsetRange(6, Long.MAX_VALUE), res.getResidual()); + + tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + assertTrue(tracker.tryClaim(5L)); + simpleEstimator.setEstimateRangeEnd(20L); + res = tracker.trySplit(0); + tracker.checkDone(); + assertEquals(new OffsetRange(0, 6), res.getPrimary()); + assertEquals(new OffsetRange(6, Long.MAX_VALUE), res.getResidual()); + } + + @Test + public void testCheckpointAfterAllProcessed() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + assertFalse(tracker.tryClaim(Long.MAX_VALUE)); + tracker.checkDone(); + assertNull(tracker.trySplit(0)); + } + + @Test + public void testCheckpointAtEmptyRange() throws Exception { + GrowableOffsetRangeTracker tracker = + new GrowableOffsetRangeTracker(Long.MAX_VALUE, new SimpleEstimator()); + tracker.checkDone(); + assertNull(tracker.trySplit(0)); + } + + @Test + public void testSplit() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + assertTrue(tracker.tryClaim(0L)); + + simpleEstimator.setEstimateRangeEnd(16L); + // The split of infinite range results in one finite range and one infinite range. + SplitResult res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(0, 8), res.getPrimary()); + assertEquals(new OffsetRange(0, 8), tracker.currentRestriction()); + assertEquals(new OffsetRange(8, Long.MAX_VALUE), res.getResidual()); + + // After the first split, the tracker should track a finite range. Estimate offset should not + // impact split. + simpleEstimator.setEstimateRangeEnd(12L); + res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(0, 4), res.getPrimary()); + assertEquals(new OffsetRange(0, 4), tracker.currentRestriction()); + assertEquals(new OffsetRange(4, 8), res.getResidual()); + assertFalse(tracker.tryClaim(4L)); + tracker.checkDone(); + } + + @Test + public void testSplitWithMaxEstimateRangeEnd() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(0L, simpleEstimator); + assertTrue(tracker.tryClaim(1L)); + simpleEstimator.setEstimateRangeEnd(Long.MAX_VALUE); + SplitResult res = tracker.trySplit(0.5); + long expectedEnd = 1L + (Long.MAX_VALUE - 1L) / 2; + assertEquals(new OffsetRange(0L, expectedEnd), res.getPrimary()); + assertEquals(new OffsetRange(expectedEnd, Long.MAX_VALUE), res.getResidual()); + } + + @Test + public void testProgressBeforeStart() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(10L, simpleEstimator); + simpleEstimator.setEstimateRangeEnd(20); + Progress currentProcess = tracker.getProgress(); + assertEquals(0, currentProcess.getWorkCompleted(), 0.001); + assertEquals(10, currentProcess.getWorkRemaining(), 0.001); + + simpleEstimator.setEstimateRangeEnd(15); + currentProcess = tracker.getProgress(); + assertEquals(0, currentProcess.getWorkCompleted(), 0.001); + assertEquals(5, currentProcess.getWorkRemaining(), 0.001); + + simpleEstimator.setEstimateRangeEnd(5); + currentProcess = tracker.getProgress(); + assertEquals(0, currentProcess.getWorkCompleted(), 0.001); + assertEquals(0, currentProcess.getWorkRemaining(), 0.001); + } + + @Test + public void testProgressAfterFinished() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(10L, simpleEstimator); + assertFalse(tracker.tryClaim(Long.MAX_VALUE)); + tracker.checkDone(); + simpleEstimator.setEstimateRangeEnd(0L); + Progress currentProgress = tracker.getProgress(); + assertEquals(Long.MAX_VALUE - 10L, currentProgress.getWorkCompleted(), 0.001); + assertEquals(0, currentProgress.getWorkRemaining(), 0.001); + } + + @Test + public void testProgress() throws Exception { + long start = 10L; + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = new GrowableOffsetRangeTracker(start, simpleEstimator); + long cur = 20L; + assertTrue(tracker.tryClaim(cur)); + + simpleEstimator.setEstimateRangeEnd(5L); + Progress currentProgress = tracker.getProgress(); + assertEquals(cur - start, currentProgress.getWorkCompleted(), 0.001); + assertEquals(0, currentProgress.getWorkRemaining(), 0.001); + + simpleEstimator.setEstimateRangeEnd(35L); + currentProgress = tracker.getProgress(); + assertEquals(cur - start, currentProgress.getWorkCompleted(), 0.001); + assertEquals(35L - cur, currentProgress.getWorkRemaining(), 0.001); + + simpleEstimator.setEstimateRangeEnd(25L); + currentProgress = tracker.getProgress(); + assertEquals(cur - start, currentProgress.getWorkCompleted(), 0.001); + assertEquals(25L - cur, currentProgress.getWorkRemaining(), 0.001); + + simpleEstimator.setEstimateRangeEnd(Long.MAX_VALUE); + currentProgress = tracker.getProgress(); + assertEquals(cur - start, currentProgress.getWorkCompleted(), 0.001); + assertEquals(Long.MAX_VALUE - cur, currentProgress.getWorkRemaining(), 0.001); + } + + @Test + public void testLargeRange() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = + new GrowableOffsetRangeTracker(Long.MIN_VALUE, simpleEstimator); + + simpleEstimator.setEstimateRangeEnd(Long.MAX_VALUE); + Progress progress = tracker.getProgress(); + assertEquals(0, progress.getWorkCompleted(), 0.001); + assertEquals( + BigDecimal.valueOf(Long.MAX_VALUE) + .subtract(BigDecimal.valueOf(Long.MIN_VALUE), MathContext.DECIMAL128) + .doubleValue(), + progress.getWorkRemaining(), + 0.001); + + simpleEstimator.setEstimateRangeEnd(Long.MIN_VALUE); + SplitResult res = tracker.trySplit(0); + assertEquals(new OffsetRange(Long.MIN_VALUE, Long.MIN_VALUE), res.getPrimary()); + assertEquals(new OffsetRange(Long.MIN_VALUE, Long.MAX_VALUE), res.getResidual()); + } + + @Test + public void testSmallRangeWithLargeValue() throws Exception { + SimpleEstimator simpleEstimator = new SimpleEstimator(); + GrowableOffsetRangeTracker tracker = + new GrowableOffsetRangeTracker(123456789012345677L, simpleEstimator); + assertTrue(tracker.tryClaim(123456789012345677L)); + simpleEstimator.setEstimateRangeEnd(123456789012345679L); + SplitResult res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(123456789012345677L, 123456789012345678L), res.getPrimary()); + assertEquals(new OffsetRange(123456789012345678L, Long.MAX_VALUE), res.getResidual()); + + tracker = new GrowableOffsetRangeTracker(123456789012345681L, simpleEstimator); + assertTrue(tracker.tryClaim(123456789012345681L)); + simpleEstimator.setEstimateRangeEnd(123456789012345683L); + res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(123456789012345681L, 123456789012345682L), res.getPrimary()); + assertEquals(new OffsetRange(123456789012345682L, Long.MAX_VALUE), res.getResidual()); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java index 6fdd2549a301..b2c381c58590 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.math.BigDecimal; +import java.math.MathContext; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.Progress; import org.junit.Rule; @@ -35,6 +37,12 @@ public class OffsetRangeTrackerTest { @Rule public final ExpectedException expected = ExpectedException.none(); + @Test + public void testIllegalInitialization() throws Exception { + expected.expect(NullPointerException.class); + OffsetRangeTracker tracker = new OffsetRangeTracker(null); + } + @Test public void testTryClaim() throws Exception { OffsetRange range = new OffsetRange(100, 200); @@ -231,4 +239,39 @@ public void testBacklogPartiallyCompleted() { assertEquals(50, progress.getWorkCompleted(), 0.001); assertEquals(50, progress.getWorkRemaining(), 0.001); } + + @Test + public void testLargeRange() throws Exception { + OffsetRangeTracker tracker = + new OffsetRangeTracker(new OffsetRange(Long.MIN_VALUE, Long.MAX_VALUE)); + + Progress progress = tracker.getProgress(); + assertEquals(0, progress.getWorkCompleted(), 0.001); + assertEquals( + BigDecimal.valueOf(Long.MAX_VALUE) + .subtract(BigDecimal.valueOf(Long.MIN_VALUE), MathContext.DECIMAL128) + .doubleValue(), + progress.getWorkRemaining(), + 0.001); + + SplitResult res = tracker.trySplit(0); + assertEquals(new OffsetRange(Long.MIN_VALUE, Long.MIN_VALUE), res.getPrimary()); + assertEquals(new OffsetRange(Long.MIN_VALUE, Long.MAX_VALUE), res.getResidual()); + } + + @Test + public void testSmallRangeWithLargeValue() throws Exception { + OffsetRangeTracker tracker = + new OffsetRangeTracker(new OffsetRange(123456789012345677L, 123456789012345679L)); + assertTrue(tracker.tryClaim(123456789012345677L)); + SplitResult res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(123456789012345677L, 123456789012345678L), res.getPrimary()); + assertEquals(new OffsetRange(123456789012345678L, 123456789012345679L), res.getResidual()); + + tracker = new OffsetRangeTracker(new OffsetRange(123456789012345681L, 123456789012345683L)); + assertTrue(tracker.tryClaim(123456789012345681L)); + res = tracker.trySplit(0.5); + assertEquals(new OffsetRange(123456789012345681L, 123456789012345682L), res.getPrimary()); + assertEquals(new OffsetRange(123456789012345682L, 123456789012345683L), res.getResidual()); + } } From cf0375689181528d55d5a611d6c6789beced9291 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 20 May 2020 16:15:26 -0700 Subject: [PATCH 039/151] [BEAM-9899] Fix some issues around storing schema `id` on user types (#11701) * Change column name to 'id' to expose row_coder error * Pull out fields as a tuple rather than attribute access Fixes bug where we read the incorrect value for a field named 'id' * Ensure user types have a _beam_schema_id set * Tag types with '_beam_schema_id' rather than 'id' to avoid collisions * lint * Revert "Pull out fields as a tuple rather than attribute access" This reverts commit 3198086a33ba44da92719bc80d989a2ca7b49d38. --- sdks/python/apache_beam/transforms/sql_test.py | 14 +++++++------- sdks/python/apache_beam/typehints/schemas.py | 11 ++++++++--- sdks/python/apache_beam/typehints/schemas_test.py | 11 +++++++++++ 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/transforms/sql_test.py b/sdks/python/apache_beam/transforms/sql_test.py index d2271bbfd923..918954cb84d7 100644 --- a/sdks/python/apache_beam/transforms/sql_test.py +++ b/sdks/python/apache_beam/transforms/sql_test.py @@ -37,10 +37,10 @@ from apache_beam.transforms.sql import SqlTransform SimpleRow = typing.NamedTuple( - "SimpleRow", [("int", int), ("str", unicode), ("flt", float)]) + "SimpleRow", [("id", int), ("str", unicode), ("flt", float)]) coders.registry.register_coder(SimpleRow, coders.RowCoder) -Enrich = typing.NamedTuple("Enrich", [("int", int), ("metadata", unicode)]) +Enrich = typing.NamedTuple("Enrich", [("id", int), ("metadata", unicode)]) coders.registry.register_coder(Enrich, coders.RowCoder) @@ -71,7 +71,7 @@ def test_generate_data(self): with TestPipeline() as p: out = p | SqlTransform( """SELECT - CAST(1 AS INT) AS `int`, + CAST(1 AS INT) AS `id`, CAST('foo' AS VARCHAR) AS `str`, CAST(3.14 AS DOUBLE) AS `flt`""") assert_that(out, equal_to([(1, "foo", 3.14)])) @@ -80,7 +80,7 @@ def test_project(self): with TestPipeline() as p: out = ( p | beam.Create([SimpleRow(1, "foo", 3.14)]) - | SqlTransform("SELECT `int`, `flt` FROM PCOLLECTION")) + | SqlTransform("SELECT `id`, `flt` FROM PCOLLECTION")) assert_that(out, equal_to([(1, 3.14)])) def test_filter(self): @@ -109,7 +109,7 @@ def test_agg(self): SELECT `str`, COUNT(*) AS `count`, - SUM(`int`) AS `sum`, + SUM(`id`) AS `sum`, AVG(`flt`) AS `avg` FROM PCOLLECTION GROUP BY `str`""")) assert_that(out, equal_to([("foo", 3, 3, 2), ("bar", 4, 8, 1.414)])) @@ -131,11 +131,11 @@ def test_tagged_join(self): | SqlTransform( """ SELECT - simple.`int` AS `int`, + simple.`id` AS `id`, enrich.metadata AS metadata FROM simple JOIN enrich - ON simple.`int` = enrich.`int`""")) + ON simple.`id` = enrich.`id`""")) assert_that(out, equal_to([(1, "a"), (26, "z"), (1, "a")])) def test_zetasql_generate_data(self): diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index faff159ed17e..f69f4756deaa 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -115,12 +115,16 @@ def get_schema_by_id(self, unique_id): float: schema_pb2.DOUBLE, }) +# Name of the attribute added to user types (existing and generated) to store +# the corresponding schema ID +_BEAM_SCHEMA_ID = "_beam_schema_id" + def typing_to_runner_api(type_): if _match_is_named_tuple(type_): schema = None - if hasattr(type_, 'id'): - schema = SCHEMA_REGISTRY.get_schema_by_id(type_.id) + if hasattr(type_, _BEAM_SCHEMA_ID): + schema = SCHEMA_REGISTRY.get_schema_by_id(getattr(type_, _BEAM_SCHEMA_ID)) if schema is None: fields = [ schema_pb2.Field( @@ -129,6 +133,7 @@ def typing_to_runner_api(type_): ] type_id = str(uuid4()) schema = schema_pb2.Schema(fields=fields, id=type_id) + setattr(type_, _BEAM_SCHEMA_ID, type_id) SCHEMA_REGISTRY.add(type_, schema) return schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=schema)) @@ -199,7 +204,7 @@ def typing_from_runner_api(fieldtype_proto): [(field.name, typing_from_runner_api(field.type)) for field in schema.fields]) - user_type.id = schema.id + setattr(user_type, _BEAM_SCHEMA_ID, schema.id) # Define a reduce function, otherwise these types can't be pickled # (See BEAM-9574) diff --git a/sdks/python/apache_beam/typehints/schemas_test.py b/sdks/python/apache_beam/typehints/schemas_test.py index 42e7198db4e2..2bf471734019 100644 --- a/sdks/python/apache_beam/typehints/schemas_test.py +++ b/sdks/python/apache_beam/typehints/schemas_test.py @@ -37,6 +37,7 @@ from apache_beam.portability.api import schema_pb2 from apache_beam.typehints.schemas import named_tuple_from_schema +from apache_beam.typehints.schemas import named_tuple_to_schema from apache_beam.typehints.schemas import typing_from_runner_api from apache_beam.typehints.schemas import typing_to_runner_api @@ -285,6 +286,16 @@ def test_generated_class_pickle(self): self.assertEqual(instance, pickle.loads(pickle.dumps(instance))) + def test_user_type_annotated_with_id_after_conversion(self): + MyCuteClass = NamedTuple('MyCuteClass', [ + ('name', unicode), + ]) + self.assertFalse(hasattr(MyCuteClass, '_beam_schema_id')) + + schema = named_tuple_to_schema(MyCuteClass) + self.assertTrue(hasattr(MyCuteClass, '_beam_schema_id')) + self.assertEqual(MyCuteClass._beam_schema_id, schema.id) + if __name__ == '__main__': unittest.main() From 3c1c757f353073ceb2947f4ad65fe7ecd56b73b6 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Wed, 20 May 2020 17:27:56 -0700 Subject: [PATCH 040/151] Merge pull request #11339 from [BEAM-9468] Fhir io * add HL7v2IO and FhirIO, test dependency issue * add to changes * minor updates * staging before splitting PRs * remove FhirIO * simplify HL7v2IO structure * add write result * refactor to HL7v2IO.Read.Result * add Coders and HL7v2IOTest * spotless * encode more of the Message * fix read result getPipeline * add adaptive throttler to write * remove unused import * improve test readability * nvm spotless apply didn't like that * scaffolding of Read integration test scaffolding of Read integration test * scaffolding for Write IT * small integration test * clean up * fix integration tests * add labels to Message coder * add parsed data coder * Refactor to use HL7v2Message wrapper class * Expose schematized data as JSON string to support motivating use case of HL7v2 -> FHIR via HCLS DataHamonization Mapping Engine. * The healthcare model Message class has fields (namely ParsedData and PatientIds) that are difficult to deal with in the Coder and provide little value for users. * set scope and user agent * add schematized data to coder * refactor list fn to avoid double get, improve tests, extract json from schematized data * add IT for new ListHL7v2Messages PTransform * migration to latest beta client library * beta api migration * add RunWith * remove adaptive throttling * revert AdaptiveThrottler to package private * Revert "remove FhirIO" This reverts commit d6967914e10da1bf42815573b567cfff564d71a1. fhir write wip wip fhirio * remove fhir adaptive throttling * spotless * fix javadoc * wip: tests failing with 400 errors * prepared integration tests * staging EOD * wip * add HealthcareIOErrorToTableRow convenience transform * spotless * wip * Add convenience function for writing deadletter to bigquery * Address review feedback * Fix FetchHL7v2Message javadoc * Add observedTime to HealthcareIOErrorCoder * Improve API surface for users * ephemeral HL7v2 stores in integration tests * prepare integration tests for apache-beam-testing * workaround broken executeBundle in client library * integration tests for FhirIO * separate ITs for all fhir versions * r4 working * remove DLQ in integration test * DRY up GCS clean up in util * manage TODO comments * improve docstring * improve docs rename tests * fix docstring typos * fhirio test * update google_clients_version to 1.30.9 * fix merge issue w/ duplicate google_oauth_clients_version * add placeholder for FhirIOReadIT * parameterize write ITs add scaffolidng for Read IT * add parameterized FhirIOReadIT * read project from options * remove FhirIO.Read experimental tag / warning * Use Jackson * complete jackson migration * spotlessApply * add TODO to add support for batch API in the future once available * add reshuffle to increase downstream paralelism * add setCoder before reshuffle * add GroupIntoBatches logic to FhirIO.Import to write more resources to files * add GroupIntoBatches test for uneven batches * test scoped pubsub topic * Refactor FhirIO.Import This commit is the first in a series of explorations of best method for batching files to a single GCS prefix for fhirStores.import call. FhirIO.Import now follow the algorithm of: - Write Bundles of input PCollection to GCS - GroupByKey (random key to parallelize GroupIntoBatches) - GroupIntoBatches (of 10,000 files) - move each batch of files to uniquely named GCS path - make import calls for each of these GCS paths * numShards 5->100 * fix auto value issue causing canot find symbol * Patches for HL7v2IO * Use TestPipeline in ITs * Drop schematized data before calling message ingest (should be output only) to help pipelines that read/write from/to two HL7v2 stores * Make HL7v2MessageCoder constructor public * use TestPipeline * block on run * add sleep to avoid flakiness due to asyncronous HL7v2 indexing * E2E integration test * fix merge issue * reconcile double sleeping * improve error hanlding * improve error handling * fix docs typo * add latency distribution metrics * remove unused imports * ingest only data and labels * fix comment * call spliterator directly, use page size 1000 * output elements more eagerly in ListHL72MessageFn * eagerly emit data from early pages * Optimization of Listing and Stablization of ITs * allow HL7v2 Message listing to emit early panes rather than waiting on pagination of all list results * add EBO on HL7v2 Message listing reaching a certain expected length in ITs to account for async indexing BEAM-9779 * revert unrelated changes * add back test * Add constant for HL7v2 indexing timeout minutes * Add constant for HL7v2 indexing timeout minutes * fix checkstyle * implement improvements in BEAM-9831 * use HL7V2_INDEXING_TIMEOUT_MINUTES everywhere * make HL7v2Message constructor public * remove dead code * add TODO for verifying triggering behavior BEAM-9847 * remove inappropriate 'optimizaiton', add status code to HealthcareIOError * refactor Fhir to use String, add status code for apache http client lib errors * fix reading * fix FHIR resource reading to use object mapper * Clean up based on offline review feedback * Add SDK version to user agent * Improve naming of FetchString * Add value provider support * Refactors Import to Copy and Clean Up * Uses copy instead of rename in FhirIO.ImportFn to be more resilient to rescheduling * adds .of() static methods for all coders * make FhirIO.Import extend FhirIO.Write and return both body and file failures * improving coder interface and safety of FhirIO.Import * check failedBodies empty in FhirIO.WriteIT * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java Co-authored-by: Pablo * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java Co-authored-by: Pablo * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java Co-authored-by: Pablo * fallback to tempLocation for fhir import temp files * spotless * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java Co-authored-by: Pablo * fixing javadoc * avoid exposing apache http dep in api * copy test resources for dataflow ITs * remove preconditions from test util * fix value provider implementation for FhirIO * wip * wip * Exclude FhirIOReadIT from google-cloud-dataflow-java tests * spotless Co-authored-by: Pablo --- .../google-cloud-dataflow-java/build.gradle | 8 +- .../sdk/transforms/GroupIntoBatchesTest.java | 51 +- .../io/google-cloud-platform/build.gradle | 4 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 1176 + .../beam/sdk/io/gcp/healthcare/HL7v2IO.java | 11 +- .../io/gcp/healthcare/HL7v2MessageCoder.java | 6 +- .../gcp/healthcare/HealthcareApiClient.java | 52 +- .../io/gcp/healthcare/HealthcareIOError.java | 9 +- .../healthcare/HttpHealthcareApiClient.java | 216 +- .../beam/sdk/io/gcp/pubsub/TestPubsub.java | 4 +- .../sdk/io/gcp/healthcare/FhirIOReadIT.java | 147 + .../sdk/io/gcp/healthcare/FhirIOTest.java | 112 + .../io/gcp/healthcare/FhirIOTestOptions.java | 46 + .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 127 + .../sdk/io/gcp/healthcare/FhirIOWriteIT.java | 129 + .../sdk/io/gcp/healthcare/HL7v2IOReadIT.java | 4 +- .../io/gcp/healthcare/HL7v2IOReadWriteIT.java | 2 +- .../sdk/io/gcp/healthcare/HL7v2IOWriteIT.java | 2 +- ..._ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json | 14018 +++ ..._1fd2683f-2a56-47c7-b674-98f2cc8319e7.json | 10877 +++ ..._4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json | 17399 ++++ ..._8ed31d3e-7352-4055-89c4-f017db3f594e.json | 9279 ++ ..._f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json | 10047 ++ .../hospitalInformation1586309771387.json | 825 + .../practitionerInformation1586309771387.json | 617 + ..._d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json | 16391 ++++ ..._af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json | 31841 ++++++ ..._a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json | 16971 ++++ ..._fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json | 80348 ++++++++++++++++ ..._55a5307d-5f23-49c1-9100-7d5c513abca3.json | 16018 +++ .../R4/hospitalInformation1586368892823.json | 1182 + .../practitionerInformation1586368892823.json | 978 + ..._02a63c07-9fcc-42ba-aec0-9d5399ac4796.json | 16003 +++ ..._88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json | 15372 +++ ..._df27a976-5c5e-4b84-ad00-fe32972dce9c.json | 22688 +++++ ..._136f997d-0a94-4573-97da-a53b5060a612.json | 28126 ++++++ ..._74779846-85a4-4b26-9da4-414a5fec1aed.json | 18231 ++++ ..._3024090f-fe14-40a9-8fae-79952d3c95ce.json | 32061 ++++++ .../hospitalInformation1586298239556.json | 880 + .../practitionerInformation1586298239556.json | 523 + 40 files changed, 362685 insertions(+), 96 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestOptions.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json create mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 0f42b9d8a7db..f5f585041707 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -223,7 +223,12 @@ task validatesRunner { dependsOn validatesRunnerLegacyWorkerTest } -task googleCloudPlatformLegacyWorkerIntegrationTest(type: Test) { +task copyGoogleCloudPlatformTestResources(type: Copy){ + from project(':sdks:java:io:google-cloud-platform').fileTree("src/test/resources") + into "$buildDir/resources/test/" +} + +task googleCloudPlatformLegacyWorkerIntegrationTest(type: Test, dependsOn: copyGoogleCloudPlatformTestResources) { group = "Verification" dependsOn ":runners:google-cloud-dataflow-java:worker:legacy-worker:shadowJar" systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ @@ -239,6 +244,7 @@ task googleCloudPlatformLegacyWorkerIntegrationTest(type: Test) { exclude '**/BigQueryIOReadIT.class' exclude '**/BigQueryIOStorageReadTableRowIT.class' exclude '**/PubsubReadIT.class' + exclude '**/FhirIOReadIT.class' maxParallelForks 4 classpath = configurations.googleCloudPlatformIntegrationTest testClassesDirs = files(project(":sdks:java:io:google-cloud-platform").sourceSets.test.output.classesDirs) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java index dab99e659a3a..ab49ec0bbcd3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java @@ -54,13 +54,14 @@ @RunWith(JUnit4.class) public class GroupIntoBatchesTest implements Serializable { private static final int BATCH_SIZE = 5; - private static final long NUM_ELEMENTS = 10; + private static final long EVEN_NUM_ELEMENTS = 10; + private static final long ODD_NUM_ELEMENTS = 11; private static final int ALLOWED_LATENESS = 0; private static final Logger LOG = LoggerFactory.getLogger(GroupIntoBatchesTest.class); @Rule public transient TestPipeline pipeline = TestPipeline.create(); - private transient ArrayList> data = createTestData(); + private transient ArrayList> data = createTestData(EVEN_NUM_ELEMENTS); - private static ArrayList> createTestData() { + private static ArrayList> createTestData(long numElements) { String[] scientists = { "Einstein", "Darwin", @@ -74,7 +75,7 @@ private static ArrayList> createTestData() { "Maxwell" }; ArrayList> data = new ArrayList<>(); - for (int i = 0; i < NUM_ELEMENTS; i++) { + for (int i = 0; i < numElements; i++) { int index = i % scientists.length; KV element = KV.of("key", scientists[index]); data.add(element); @@ -111,7 +112,45 @@ public Void apply(Iterable>> input) { } }); PAssert.thatSingleton("Incorrect collection size", collection.apply("Count", Count.globally())) - .isEqualTo(NUM_ELEMENTS / BATCH_SIZE); + .isEqualTo(EVEN_NUM_ELEMENTS / BATCH_SIZE); + pipeline.run(); + } + + /** test behavior when the number of input elements is not evenly divisible by batch size. */ + @Test + @Category({NeedsRunner.class, UsesTimersInParDo.class, UsesStatefulParDo.class}) + public void testWithUnevenBatches() { + PCollection>> collection = + pipeline + .apply("Input data", Create.of(createTestData(ODD_NUM_ELEMENTS))) + .apply(GroupIntoBatches.ofSize(BATCH_SIZE)) + // set output coder + .setCoder(KvCoder.of(StringUtf8Coder.of(), IterableCoder.of(StringUtf8Coder.of()))); + PAssert.that("Incorrect batch size in one or more elements", collection) + .satisfies( + new SerializableFunction>>, Void>() { + + private boolean checkBatchSizes(Iterable>> listToCheck) { + for (KV> element : listToCheck) { + // number of elements should be less than or equal to BATCH_SIZE + if (Iterables.size(element.getValue()) > BATCH_SIZE) { + return false; + } + } + return true; + } + + @Override + public Void apply(Iterable>> input) { + assertTrue(checkBatchSizes(input)); + return null; + } + }); + PAssert.thatSingleton("Incorrect collection size", collection.apply("Count", Count.globally())) + .isEqualTo( + // round up division for positive numbers + // https://math.stackexchange.com/questions/2591316/proof-for-integer-division-algorithm-that-rounds-up. + (ODD_NUM_ELEMENTS + BATCH_SIZE - 1) / BATCH_SIZE); pipeline.run(); } @@ -142,7 +181,7 @@ public void testInStreamingMode() { streamBuilder .advanceWatermarkTo(startInstant.plus(Duration.standardSeconds(windowDuration - 1))) .advanceWatermarkTo(startInstant.plus(Duration.standardSeconds(windowDuration + 1))) - .advanceWatermarkTo(startInstant.plus(Duration.standardSeconds(NUM_ELEMENTS))) + .advanceWatermarkTo(startInstant.plus(Duration.standardSeconds(EVEN_NUM_ELEMENTS))) .advanceWatermarkToInfinity(); PCollection> inputCollection = diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index c6c8e93927a2..442479563f69 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -92,10 +92,11 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 } + /** * These are integration tests with the real Pubsub service and the DirectRunner. */ -task integrationTest(type: Test) { +task integrationTest(type: Test, dependsOn: processTestResources) { group = "Verification" def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests' @@ -117,6 +118,7 @@ task integrationTest(type: Test) { maxParallelForks 4 classpath = sourceSets.test.runtimeClasspath testClassesDirs = sourceSets.test.output.classesDirs + useJUnit { excludeCategories "org.apache.beam.sdk.testing.UsesKms" } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java new file mode 100644 index 000000000000..13b696edd74f --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -0,0 +1,1176 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.healthcare.v1beta1.model.HttpBody; +import com.google.api.services.healthcare.v1beta1.model.Operation; +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.TextualIntegerCoder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.io.fs.MoveOptions.StandardMoveOptions; +import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.io.fs.ResourceIdCoder; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupIntoBatches; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Wait; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.codehaus.jackson.JsonProcessingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link FhirIO} provides an API for reading and writing resources to Google Cloud Healthcare Fhir API. + * + * + *

    Reading

    + * + *

    FHIR resources can be read with {@link FhirIO.Read}, which supports use cases where you have a + * ${@link PCollection} of message IDS. This is appropriate for reading the Fhir notifications from + * a Pub/Sub subscription with {@link PubsubIO#readStrings()} or in cases where you have a manually + * prepared list of messages that you need to process (e.g. in a text file read with {@link + * org.apache.beam.sdk.io.TextIO}*) . + * + *

    Fetch Resource contents from Fhir Store based on the {@link PCollection} of message ID strings + * {@link FhirIO.Read.Result} where one can call {@link Read.Result#getResources()} to retrieve a + * {@link PCollection} containing the successfully fetched {@link String}s and/or {@link + * FhirIO.Read.Result#getFailedReads()}* to retrieve a {@link PCollection} of {@link + * HealthcareIOError}* containing the resource ID that could not be fetched and the exception as a + * {@link HealthcareIOError}, this can be used to write to the dead letter storage system of your + * choosing. This error handling is mainly to transparently surface errors where the upstream {@link + * PCollection}* contains IDs that are not valid or are not reachable due to permissions issues. + * + *

    Writing

    + * + *

    Write Resources can be written to FHIR with two different methods: Import or Execute Bundle. + * + *

    Execute Bundle This is best for use cases where you are writing to a non-empty FHIR store with + * other clients or otherwise need referential integrity (e.g. A Streaming HL7v2 to FHIR ETL + * pipeline). + * + *

    Import This is best for use cases where you are populating an empty FHIR store with no other + * clients. It is faster than the execute bundles method but does not respect referential integrity + * and the resources are not written transactionally (e.g. a historicaly backfill on a new FHIR + * store) This requires each resource to contain a client provided ID. It is important that when + * using import you give the appropriate permissions to the Google Cloud Healthcare Service Agent. + * + * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/executeBundle> + * @see https://cloud.google.com/healthcare/docs/how-tos/permissions-healthcare-api-gcp-products#fhir_store_cloud_storage_permissions> + * @see https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores/import> + * A {@link PCollection} of {@link String} can be ingested into an Fhir store using {@link + * FhirIO.Write#fhirStoresImport(String, String, String, FhirIO.Import.ContentStructure)} This + * will return a {@link FhirIO.Write.Result} on which you can call {@link + * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link + * HealthcareIOError} containing the {@link String} that failed to be ingested and the + * exception. + *

    Example + *

    {@code
    + * Pipeline pipeline = ...
    + *
    + * // Tail the FHIR store by retrieving resources based on Pub/Sub notifications.
    + * FhirIO.Read.Result readResult = p
    + *   .apply("Read FHIR notifications",
    + *     PubsubIO.readStrings().fromSubscription(options.getNotificationSubscription()))
    + *   .apply(FhirIO.readResources());
    + *
    + * // happily retrived messages
    + * PCollection resources = readResult.getResources();
    + * // message IDs that couldn't be retrieved + error context
    + * PCollection> failedReads = readResult.getFailedReads();
    + *
    + * failedReads.apply("Write Message IDs / Stacktrace for Failed Reads to BigQuery",
    + *     BigQueryIO
    + *         .write()
    + *         .to(option.getBQFhirExecuteBundlesDeadLetterTable())
    + *         .withFormatFunction(new HealthcareIOErrorToTableRow()));
    + *
    + * output = resources.apply("Happy path transformations", ...);
    + * FhirIO.Write.Result writeResult =
    + *     output.apply("Execute FHIR Bundles", FhirIO.executeBundles(options.getExistingFhirStore()));
    + *
    + * PCollection> failedBundles = writeResult.getFailedInsertsWithErr();
    + *
    + * failedBundles.apply("Write failed bundles to BigQuery",
    + *     BigQueryIO
    + *         .write()
    + *         .to(option.getBQFhirExecuteBundlesDeadLetterTable())
    + *         .withFormatFunction(new HealthcareIOErrorToTableRow()));
    + *
    + * // Alternatively you could use import for high throughput to a new store.
    + * FhirIO.Write.Result writeResult =
    + *     output.apply("Import FHIR Resources", FhirIO.executeBundles(options.getNewFhirStore()));
    + * }***
    + * 
    + */ +public class FhirIO { + + /** + * Read resources from a PCollection of resource IDs (e.g. when subscribing the pubsub + * notifications) + * + * @return the read + * @see Read + */ + public static Read readResources() { + return new Read(); + } + + /** + * Import resources. Intended for use on empty FHIR stores + * + * @param fhirStore the fhir store + * @param tempDir the temp dir + * @param deadLetterDir the dead letter dir + * @param contentStructure the content structure + * @return the import + * @see Import + */ + public static Import importResources( + String fhirStore, + String tempDir, + String deadLetterDir, + @Nullable FhirIO.Import.ContentStructure contentStructure) { + return new Import(fhirStore, tempDir, deadLetterDir, contentStructure); + } + + /** + * Import resources. Intended for use on empty FHIR stores + * + * @param fhirStore the fhir store + * @param tempDir the temp dir + * @param deadLetterDir the dead letter dir + * @param contentStructure the content structure + * @return the import + * @see Import + */ + public static Import importResources( + ValueProvider fhirStore, + ValueProvider tempDir, + ValueProvider deadLetterDir, + @Nullable FhirIO.Import.ContentStructure contentStructure) { + return new Import(fhirStore, tempDir, deadLetterDir, contentStructure); + } + + /** The type Read. */ + public static class Read extends PTransform, FhirIO.Read.Result> { + private static final Logger LOG = LoggerFactory.getLogger(Read.class); + + /** Instantiates a new Read. */ + public Read() {} + + /** The type Result. */ + public static class Result implements POutput, PInput { + private PCollection resources; + + private PCollection> failedReads; + /** The Pct. */ + PCollectionTuple pct; + + /** + * Create FhirIO.Read.Result form PCollectionTuple with OUT and DEAD_LETTER tags. + * + * @param pct the pct + * @return the read result + * @throws IllegalArgumentException the illegal argument exception + */ + static FhirIO.Read.Result of(PCollectionTuple pct) throws IllegalArgumentException { + if (pct.getAll() + .keySet() + .containsAll((Collection) TupleTagList.of(OUT).and(DEAD_LETTER))) { + return new FhirIO.Read.Result(pct); + } else { + throw new IllegalArgumentException( + "The PCollection tuple must have the FhirIO.Read.OUT " + + "and FhirIO.Read.DEAD_LETTER tuple tags"); + } + } + + private Result(PCollectionTuple pct) { + this.pct = pct; + this.resources = pct.get(OUT); + this.failedReads = + pct.get(DEAD_LETTER).setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of())); + } + + /** + * Gets failed reads. + * + * @return the failed reads + */ + public PCollection> getFailedReads() { + return failedReads; + } + + /** + * Gets resources. + * + * @return the resources + */ + public PCollection getResources() { + return resources; + } + + @Override + public Pipeline getPipeline() { + return this.pct.getPipeline(); + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of(OUT, resources); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } + + /** The tag for the main output of Fhir Messages. */ + public static final TupleTag OUT = new TupleTag() {}; + /** The tag for the deadletter output of Fhir Messages. */ + public static final TupleTag> DEAD_LETTER = + new TupleTag>() {}; + + @Override + public FhirIO.Read.Result expand(PCollection input) { + return input.apply("Fetch Fhir messages", new FetchResourceJsonString()); + } + + /** + * DoFn to fetch a resource from an Google Cloud Healthcare FHIR store based on resourceID + * + *

    This DoFn consumes a {@link PCollection} of notifications {@link String}s from the FHIR + * store, and fetches the actual {@link String} object based on the id in the notification and + * will output a {@link PCollectionTuple} which contains the output and dead-letter {@link + * PCollection}*. + * + *

    The {@link PCollectionTuple} output will contain the following {@link PCollection}: + * + *

      + *
    • {@link FhirIO.Read#OUT} - Contains all {@link PCollection} records successfully read + * from the Fhir store. + *
    • {@link FhirIO.Read#DEAD_LETTER} - Contains all {@link PCollection} of {@link + * HealthcareIOError}* of message IDs which failed to be fetched from the Fhir store, with + * error message and stacktrace. + *
    + */ + static class FetchResourceJsonString + extends PTransform, FhirIO.Read.Result> { + + /** Instantiates a new Fetch Fhir message DoFn. */ + public FetchResourceJsonString() {} + + @Override + public FhirIO.Read.Result expand(PCollection resourceIds) { + return new FhirIO.Read.Result( + resourceIds.apply( + ParDo.of(new ReadResourceFn()) + .withOutputTags(FhirIO.Read.OUT, TupleTagList.of(FhirIO.Read.DEAD_LETTER)))); + } + + /** DoFn for fetching messages from the Fhir store with error handling. */ + static class ReadResourceFn extends DoFn { + + private Counter failedMessageGets = + Metrics.counter(ReadResourceFn.class, "failed-message-reads"); + private static final Logger LOG = LoggerFactory.getLogger(ReadResourceFn.class); + private final Counter successfulStringGets = + Metrics.counter(ReadResourceFn.class, "successful-hl7v2-message-gets"); + private HealthcareApiClient client; + private ObjectMapper mapper; + + /** Instantiates a new Hl 7 v 2 message get fn. */ + ReadResourceFn() {} + + /** + * Instantiate healthcare client. + * + * @throws IOException the io exception + */ + @Setup + public void instantiateHealthcareClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + this.mapper = new ObjectMapper(); + } + + /** + * Process element. + * + * @param context the context + */ + @ProcessElement + public void processElement(ProcessContext context) { + String resourceId = context.element(); + try { + context.output(fetchResource(this.client, resourceId)); + } catch (Exception e) { + failedMessageGets.inc(); + LOG.warn( + String.format( + "Error fetching Fhir message with ID %s writing to Dead Letter " + + "Queue. Cause: %s Stack Trace: %s", + resourceId, e.getMessage(), Throwables.getStackTraceAsString(e))); + context.output(FhirIO.Read.DEAD_LETTER, HealthcareIOError.of(resourceId, e)); + } + } + + private String fetchResource(HealthcareApiClient client, String resourceId) + throws IOException, IllegalArgumentException { + long startTime = System.currentTimeMillis(); + + HttpBody resource = client.readFhirResource(resourceId); + + if (resource == null) { + throw new IOException(String.format("GET request for %s returned null", resourceId)); + } + this.successfulStringGets.inc(); + return mapper.writeValueAsString(resource); + } + } + } + } + + /** The type Write. */ + @AutoValue + public abstract static class Write extends PTransform, Write.Result> { + + /** The tag for the failed writes to FHIR store`. */ + public static final TupleTag> FAILED_BODY = + new TupleTag>() {}; + /** The tag for the files that failed to FHIR store`. */ + public static final TupleTag> FAILED_FILES = + new TupleTag>() {}; + /** The tag for temp files for import to FHIR store`. */ + public static final TupleTag TEMP_FILES = new TupleTag() {}; + + /** The enum Write method. */ + public enum WriteMethod { + /** + * Execute Bundle Method executes a batch of requests as a single transaction @see . + */ + EXECUTE_BUNDLE, + /** + * Import Method bulk imports resources from GCS. This is ideal for initial loads to empty + * FHIR stores. . + */ + IMPORT + } + + /** The type Result. */ + public static class Result implements POutput { + private final Pipeline pipeline; + private final PCollection> failedBodies; + private final PCollection> failedFiles; + + /** + * Creates a {@link FhirIO.Write.Result} in the given {@link Pipeline}. @param pipeline the + * pipeline + * + * @param failedBodies the failed inserts + * @return the result + */ + static Result in(Pipeline pipeline, PCollection> failedBodies) { + return new Result(pipeline, failedBodies, null); + } + + static Result in( + Pipeline pipeline, + PCollection> failedBodies, + PCollection> failedFiles) { + return new Result(pipeline, failedBodies, failedFiles); + } + + /** + * Gets failed bodies with err. + * + * @return the failed inserts with err + */ + public PCollection> getFailedBodies() { + return this.failedBodies; + } + + /** + * Gets failed file imports with err. + * + * @return the failed GCS uri with err + */ + public PCollection> getFailedFiles() { + return this.failedFiles; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of(Write.FAILED_BODY, failedBodies, Write.FAILED_FILES, failedFiles); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + + private Result( + Pipeline pipeline, + PCollection> failedBodies, + @Nullable PCollection> failedFiles) { + this.pipeline = pipeline; + this.failedBodies = failedBodies; + if (failedFiles == null) { + failedFiles = + (PCollection>) + pipeline.apply(Create.empty(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + } + this.failedFiles = failedFiles; + } + } + + /** + * Gets Fhir store. + * + * @return the Fhir store + */ + abstract ValueProvider getFhirStore(); + + /** + * Gets write method. + * + * @return the write method + */ + abstract WriteMethod getWriteMethod(); + + /** + * Gets content structure. + * + * @return the content structure + */ + abstract Optional getContentStructure(); + + /** + * Gets import gcs temp path. + * + * @return the import gcs temp path + */ + abstract Optional> getImportGcsTempPath(); + + /** + * Gets import gcs dead letter path. + * + * @return the import gcs dead letter path + */ + abstract Optional> getImportGcsDeadLetterPath(); + + /** The type Builder. */ + @AutoValue.Builder + abstract static class Builder { + + /** + * Sets Fhir store. + * + * @param fhirStore the Fhir store + * @return the Fhir store + */ + abstract Builder setFhirStore(ValueProvider fhirStore); + + /** + * Sets write method. + * + * @param writeMethod the write method + * @return the write method + */ + abstract Builder setWriteMethod(WriteMethod writeMethod); + + /** + * Sets content structure. + * + * @param contentStructure the content structure + * @return the content structure + */ + abstract Builder setContentStructure(FhirIO.Import.ContentStructure contentStructure); + + /** + * Sets import gcs temp path. + * + * @param gcsTempPath the gcs temp path + * @return the import gcs temp path + */ + abstract Builder setImportGcsTempPath(ValueProvider gcsTempPath); + + /** + * Sets import gcs dead letter path. + * + * @param gcsDeadLetterPath the gcs dead letter path + * @return the import gcs dead letter path + */ + abstract Builder setImportGcsDeadLetterPath(ValueProvider gcsDeadLetterPath); + + /** + * Build write. + * + * @return the write + */ + abstract Write build(); + } + + private static Write.Builder write(String fhirStore) { + return new AutoValue_FhirIO_Write.Builder().setFhirStore(StaticValueProvider.of(fhirStore)); + } + + /** + * Create Method creates a single FHIR resource. @see + * + * @param fhirStore the hl 7 v 2 store + * @param gcsTempPath the gcs temp path + * @param gcsDeadLetterPath the gcs dead letter path + * @param contentStructure the content structure + * @return the write + */ + public static Write fhirStoresImport( + String fhirStore, + String gcsTempPath, + String gcsDeadLetterPath, + @Nullable FhirIO.Import.ContentStructure contentStructure) { + return new AutoValue_FhirIO_Write.Builder() + .setFhirStore(StaticValueProvider.of(fhirStore)) + .setWriteMethod(Write.WriteMethod.IMPORT) + .setContentStructure(contentStructure) + .setImportGcsTempPath(StaticValueProvider.of(gcsTempPath)) + .setImportGcsDeadLetterPath(StaticValueProvider.of(gcsDeadLetterPath)) + .build(); + } + + public static Write fhirStoresImport( + String fhirStore, + String gcsDeadLetterPath, + @Nullable FhirIO.Import.ContentStructure contentStructure) { + return new AutoValue_FhirIO_Write.Builder() + .setFhirStore(StaticValueProvider.of(fhirStore)) + .setWriteMethod(Write.WriteMethod.IMPORT) + .setContentStructure(contentStructure) + .setImportGcsDeadLetterPath(StaticValueProvider.of(gcsDeadLetterPath)) + .build(); + } + + public static Write fhirStoresImport( + ValueProvider fhirStore, + ValueProvider gcsTempPath, + ValueProvider gcsDeadLetterPath, + @Nullable FhirIO.Import.ContentStructure contentStructure) { + return new AutoValue_FhirIO_Write.Builder() + .setFhirStore(fhirStore) + .setWriteMethod(Write.WriteMethod.IMPORT) + .setContentStructure(contentStructure) + .setImportGcsTempPath(gcsTempPath) + .setImportGcsDeadLetterPath(gcsDeadLetterPath) + .build(); + } + + /** + * Execute Bundle Method executes a batch of requests as a single transaction @see . + * + * @param fhirStore the hl 7 v 2 store + * @return the write + */ + public static Write executeBundles(String fhirStore) { + return new AutoValue_FhirIO_Write.Builder() + .setFhirStore(StaticValueProvider.of(fhirStore)) + .setWriteMethod(WriteMethod.EXECUTE_BUNDLE) + .build(); + } + + /** + * Execute bundles write. + * + * @param fhirStore the fhir store + * @return the write + */ + public static Write executeBundles(ValueProvider fhirStore) { + return new AutoValue_FhirIO_Write.Builder() + .setFhirStore(fhirStore) + .setWriteMethod(WriteMethod.EXECUTE_BUNDLE) + .build(); + } + + private static final Logger LOG = LoggerFactory.getLogger(Write.class); + + @Override + public Result expand(PCollection input) { + PCollection> failedBundles; + PCollection> failedImports; + switch (this.getWriteMethod()) { + case IMPORT: + LOG.warn( + "Make sure the Cloud Healthcare Service Agent has permissions when using import:" + + " https://cloud.google.com/healthcare/docs/how-tos/permissions-healthcare-api-gcp-products#fhir_store_cloud_storage_permissions"); + ValueProvider deadPath = + getImportGcsDeadLetterPath().orElseThrow(IllegalArgumentException::new); + FhirIO.Import.ContentStructure contentStructure = + getContentStructure().orElseThrow(IllegalArgumentException::new); + ValueProvider tempPath = + getImportGcsTempPath() + .orElse( + StaticValueProvider.of(input.getPipeline().getOptions().getTempLocation())); + + return input.apply(new Import(getFhirStore(), tempPath, deadPath, contentStructure)); + case EXECUTE_BUNDLE: + default: + failedBundles = + input + .apply( + "Execute FHIR Bundles", + ParDo.of(new ExecuteBundles.ExecuteBundlesFn(this.getFhirStore()))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of())); + } + return Result.in(input.getPipeline(), failedBundles); + } + } + + /** + * Writes each bundle of elements to a new-line delimited JSON file on GCS and issues a + * fhirStores.import Request for that file. This is intended for batch use only to facilitate + * large backfills to empty FHIR stores and should not be used with unbounded PCollections. If + * your use case is streaming checkout using {@link ExecuteBundles} to more safely execute bundles + * as transactions which is safer practice for a use on a "live" FHIR store. + */ + public static class Import extends Write { + + private final ValueProvider fhirStore; + private final ValueProvider deadLetterGcsPath; + private final ContentStructure contentStructure; + private static final int DEFAULT_FILES_PER_BATCH = 10000; + private static final Logger LOG = LoggerFactory.getLogger(Import.class); + private ValueProvider tempGcsPath; + + /* + * Instantiates a new Import. + * + * @param fhirStore the fhir store + * @param tempGcsPath the temp gcs path + * @param deadLetterGcsPath the dead letter gcs path + * @param contentStructure the content structure + */ + Import( + ValueProvider fhirStore, + ValueProvider tempGcsPath, + ValueProvider deadLetterGcsPath, + @Nullable ContentStructure contentStructure) { + this.fhirStore = fhirStore; + this.tempGcsPath = tempGcsPath; + this.deadLetterGcsPath = deadLetterGcsPath; + if (contentStructure == null) { + this.contentStructure = ContentStructure.CONTENT_STRUCTURE_UNSPECIFIED; + } else { + this.contentStructure = contentStructure; + } + } + + Import( + ValueProvider fhirStore, + ValueProvider deadLetterGcsPath, + @Nullable ContentStructure contentStructure) { + this.fhirStore = fhirStore; + this.deadLetterGcsPath = deadLetterGcsPath; + if (contentStructure == null) { + this.contentStructure = ContentStructure.CONTENT_STRUCTURE_UNSPECIFIED; + } else { + this.contentStructure = contentStructure; + } + } + /** + * Instantiates a new Import. + * + * @param fhirStore the fhir store + * @param tempGcsPath the temp gcs path + * @param deadLetterGcsPath the dead letter gcs path + * @param contentStructure the content structure + */ + Import( + String fhirStore, + String tempGcsPath, + String deadLetterGcsPath, + @Nullable ContentStructure contentStructure) { + this.fhirStore = StaticValueProvider.of(fhirStore); + this.tempGcsPath = StaticValueProvider.of(tempGcsPath); + this.deadLetterGcsPath = StaticValueProvider.of(deadLetterGcsPath); + if (contentStructure == null) { + this.contentStructure = ContentStructure.CONTENT_STRUCTURE_UNSPECIFIED; + } else { + this.contentStructure = contentStructure; + } + } + + @Override + ValueProvider getFhirStore() { + return fhirStore; + } + + @Override + WriteMethod getWriteMethod() { + return WriteMethod.IMPORT; + } + + @Override + Optional getContentStructure() { + return Optional.of(contentStructure); + } + + @Override + Optional> getImportGcsTempPath() { + return Optional.of(tempGcsPath); + } + + @Override + Optional> getImportGcsDeadLetterPath() { + return Optional.of(deadLetterGcsPath); + } + + @Override + public Write.Result expand(PCollection input) { + checkState( + input.isBounded() == IsBounded.BOUNDED, + "FhirIO.Import should only be used on unbounded PCollections as it is" + + "intended for batch use only."); + + // fall back on pipeline's temp location. + ValueProvider tempPath = + getImportGcsTempPath() + .orElse(StaticValueProvider.of(input.getPipeline().getOptions().getTempLocation())); + + // Write bundles of String to GCS + PCollectionTuple writeTmpFileResults = + input.apply( + "Write nd json to GCS", + ParDo.of(new WriteBundlesToFilesFn(fhirStore, tempPath, deadLetterGcsPath)) + .withOutputTags(Write.TEMP_FILES, TupleTagList.of(Write.FAILED_BODY))); + + PCollection> failedBodies = + writeTmpFileResults + .get(Write.FAILED_BODY) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of())); + int numShards = 100; + PCollection> failedFiles = + writeTmpFileResults + .get(Write.TEMP_FILES) + .apply( + "Shard files", // to paralelize group into batches + WithKeys.of(elm -> ThreadLocalRandom.current().nextInt(0, numShards))) + .setCoder(KvCoder.of(TextualIntegerCoder.of(), ResourceIdCoder.of())) + .apply("Assemble File Batches", GroupIntoBatches.ofSize(DEFAULT_FILES_PER_BATCH)) + .setCoder( + KvCoder.of(TextualIntegerCoder.of(), IterableCoder.of(ResourceIdCoder.of()))) + .apply( + "Import Batches", + ParDo.of(new ImportFn(fhirStore, tempPath, deadLetterGcsPath, contentStructure))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of())); + + input + .getPipeline() + .apply("Instantiate Temp Path", Create.ofProvider(tempPath, StringUtf8Coder.of())) + .apply( + "Resolve SubDirs", + MapElements.into(TypeDescriptors.strings()) + .via((String path) -> path.endsWith("/") ? path + "*" : path + "/*")) + .apply("Wait On File Writing", Wait.on(failedBodies)) + .apply("Wait On FHIR Importing", Wait.on(failedFiles)) + .apply( + "Match tempGcsPath", + FileIO.matchAll().withEmptyMatchTreatment(EmptyMatchTreatment.ALLOW)) + .apply( + "Delete tempGcsPath", + ParDo.of( + new DoFn() { + @ProcessElement + public void delete(@Element Metadata path, ProcessContext context) { + String tempPath = + getImportGcsTempPath() + .orElse( + StaticValueProvider.of( + context.getPipelineOptions().getTempLocation())) + .get(); + // Wait til window closes for failedBodies and failedFiles to ensure we are + // done processing + // anything under tempGcsPath because it has been successfully imported to + // FHIR store or + // copies have been moved to the dead letter path. + // Clean up all of tempGcsPath. This will handle removing phantom temporary + // objects from + // failed / rescheduled ImportFn::importBatch. + try { + FileSystems.delete( + Collections.singleton(path.resourceId()), + StandardMoveOptions.IGNORE_MISSING_FILES); + } catch (IOException e) { + LOG.error("error cleaning up tempGcsDir: %s", e); + } + } + })) + .setCoder(VoidCoder.of()); + + return Write.Result.in(input.getPipeline(), failedBodies, failedFiles); + } + + /** The Write bundles to new line delimited json files. */ + static class WriteBundlesToFilesFn extends DoFn { + + private final ValueProvider fhirStore; + private final ValueProvider tempGcsPath; + private final ValueProvider deadLetterGcsPath; + private ObjectMapper mapper; + private ResourceId resourceId; + private WritableByteChannel ndJsonChannel; + private BoundedWindow window; + + private transient HealthcareApiClient client; + private static final Logger LOG = LoggerFactory.getLogger(WriteBundlesToFilesFn.class); + + WriteBundlesToFilesFn( + ValueProvider fhirStore, + ValueProvider tempGcsPath, + ValueProvider deadLetterGcsPath) { + this.fhirStore = fhirStore; + this.tempGcsPath = tempGcsPath; + this.deadLetterGcsPath = deadLetterGcsPath; + } + + /** + * Instantiates a new Import fn. + * + * @param fhirStore the fhir store + * @param tempGcsPath the temp gcs path + * @param deadLetterGcsPath the dead letter gcs path + */ + WriteBundlesToFilesFn(String fhirStore, String tempGcsPath, String deadLetterGcsPath) { + this.fhirStore = StaticValueProvider.of(fhirStore); + this.tempGcsPath = StaticValueProvider.of(tempGcsPath); + this.deadLetterGcsPath = StaticValueProvider.of(deadLetterGcsPath); + } + + /** + * Init client. + * + * @throws IOException the io exception + */ + @Setup + public void initClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + } + + /** + * Init batch. + * + * @throws IOException the io exception + */ + @StartBundle + public void initFile() throws IOException { + // Write each bundle to newline delimited JSON file. + String filename = String.format("fhirImportBatch-%s.ndjson", UUID.randomUUID().toString()); + ResourceId tempDir = FileSystems.matchNewResource(this.tempGcsPath.get(), true); + this.resourceId = tempDir.resolve(filename, StandardResolveOptions.RESOLVE_FILE); + this.ndJsonChannel = FileSystems.create(resourceId, "application/ld+json"); + if (mapper == null) { + this.mapper = new ObjectMapper(); + } + } + + /** + * Add to batch. + * + * @param context the context + * @throws IOException the io exception + */ + @ProcessElement + public void addToFile(ProcessContext context, BoundedWindow window) throws IOException { + this.window = window; + String httpBody = context.element(); + try { + // This will error if not valid JSON an convert Pretty JSON to raw JSON. + Object data = this.mapper.readValue(httpBody, Object.class); + String ndJson = this.mapper.writeValueAsString(data) + "\n"; + this.ndJsonChannel.write(ByteBuffer.wrap(ndJson.getBytes(StandardCharsets.UTF_8))); + } catch (JsonProcessingException e) { + String resource = + String.format( + "Failed to parse payload: %s as json at: %s : %s." + + "Dropping message from batch import.", + httpBody.toString(), e.getLocation().getCharOffset(), e.getMessage()); + LOG.warn(resource); + context.output( + Write.FAILED_BODY, HealthcareIOError.of(httpBody, new IOException(resource))); + } + } + + /** + * Close file. + * + * @param context the context + * @throws IOException the io exception + */ + @FinishBundle + public void closeFile(FinishBundleContext context) throws IOException { + // Write the file with all elements in this bundle to GCS. + ndJsonChannel.close(); + context.output(resourceId, window.maxTimestamp(), window); + } + } + + /** Import batches of new line delimited json files to FHIR Store. */ + static class ImportFn + extends DoFn>, HealthcareIOError> { + + private static final Logger LOG = LoggerFactory.getLogger(ImportFn.class); + private final ValueProvider tempGcsPath; + private final ValueProvider deadLetterGcsPath; + private ResourceId tempDir; + private final ContentStructure contentStructure; + private HealthcareApiClient client; + private final ValueProvider fhirStore; + + ImportFn( + ValueProvider fhirStore, + ValueProvider tempGcsPath, + ValueProvider deadLetterGcsPath, + @Nullable ContentStructure contentStructure) { + this.fhirStore = fhirStore; + this.tempGcsPath = tempGcsPath; + this.deadLetterGcsPath = deadLetterGcsPath; + if (contentStructure == null) { + this.contentStructure = ContentStructure.CONTENT_STRUCTURE_UNSPECIFIED; + } else { + this.contentStructure = contentStructure; + } + } + + @Setup + public void init() throws IOException { + tempDir = + FileSystems.matchNewResource(tempGcsPath.get(), true) + .resolve( + String.format("tmp-%s", UUID.randomUUID().toString()), + StandardResolveOptions.RESOLVE_DIRECTORY); + client = new HttpHealthcareApiClient(); + } + + /** + * Move files to a temporary subdir (to provide common prefix) to execute import with single + * GCS URI. + */ + @ProcessElement + public void importBatch( + @Element KV> element, + OutputReceiver> output) + throws IOException { + Iterable batch = element.getValue(); + List tempDestinations = new ArrayList<>(); + List deadLetterDestinations = new ArrayList<>(); + assert batch != null; + for (ResourceId file : batch) { + tempDestinations.add( + tempDir.resolve(file.getFilename(), StandardResolveOptions.RESOLVE_FILE)); + deadLetterDestinations.add( + FileSystems.matchNewResource(deadLetterGcsPath.get(), true) + .resolve(file.getFilename(), StandardResolveOptions.RESOLVE_FILE)); + } + FileSystems.copy(ImmutableList.copyOf(batch), tempDestinations); + ResourceId importUri = tempDir.resolve("*", StandardResolveOptions.RESOLVE_FILE); + try { + // Blocking fhirStores.import request. + assert contentStructure != null; + Operation operation = + client.importFhirResource( + fhirStore.get(), importUri.toString(), contentStructure.name()); + client.pollOperation(operation, 500L); + // Clean up temp files on GCS as they we successfully imported to FHIR store and no longer + // needed. + FileSystems.delete(tempDestinations); + } catch (IOException | InterruptedException e) { + ResourceId deadLetterResourceId = + FileSystems.matchNewResource(deadLetterGcsPath.get(), true); + LOG.warn( + String.format( + "Failed to import %s with error: %s. Moving to deadletter path %s", + importUri.toString(), e.getMessage(), deadLetterResourceId.toString())); + FileSystems.rename(tempDestinations, deadLetterDestinations); + output.output(HealthcareIOError.of(importUri.toString(), e)); + } finally { + // If we've reached this point files have either been successfully import to FHIR store + // or moved to Dead Letter Queue. + // Clean up original files for this batch on GCS. + FileSystems.delete(ImmutableList.copyOf(batch)); + } + } + } + + /** The enum Content structure. */ + public enum ContentStructure { + /** If the content structure is not specified, the default value BUNDLE will be used. */ + CONTENT_STRUCTURE_UNSPECIFIED, + /** + * The source file contains one or more lines of newline-delimited JSON (ndjson). Each line is + * a bundle, which contains one or more resources. Set the bundle type to history to import + * resource versions. + */ + BUNDLE, + /** + * The source file contains one or more lines of newline-delimited JSON (ndjson). Each line is + * a single resource. + */ + RESOURCE, + /** The entire file is one JSON bundle. The JSON can span multiple lines. */ + BUNDLE_PRETTY, + /** The entire file is one JSON resource. The JSON can span multiple lines. */ + RESOURCE_PRETTY + } + } + + /** The type Execute bundles. */ + public static class ExecuteBundles extends PTransform, Write.Result> { + private final ValueProvider fhirStore; + + /** + * Instantiates a new Execute bundles. + * + * @param fhirStore the fhir store + */ + ExecuteBundles(ValueProvider fhirStore) { + this.fhirStore = fhirStore; + } + + /** + * Instantiates a new Execute bundles. + * + * @param fhirStore the fhir store + */ + ExecuteBundles(String fhirStore) { + this.fhirStore = StaticValueProvider.of(fhirStore); + } + + @Override + public FhirIO.Write.Result expand(PCollection input) { + return Write.Result.in( + input.getPipeline(), + input + .apply(ParDo.of(new ExecuteBundlesFn(fhirStore))) + .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); + } + + /** The type Write Fhir fn. */ + static class ExecuteBundlesFn extends DoFn> { + + private Counter failedBundles = Metrics.counter(ExecuteBundlesFn.class, "failed-bundles"); + private transient HealthcareApiClient client; + private final ObjectMapper mapper = new ObjectMapper(); + /** The Fhir store. */ + private final ValueProvider fhirStore; + + /** + * Instantiates a new Write Fhir fn. + * + * @param fhirStore the Fhir store + */ + ExecuteBundlesFn(ValueProvider fhirStore) { + this.fhirStore = fhirStore; + } + + /** + * Initialize healthcare client. + * + * @throws IOException the io exception + */ + @Setup + public void initClient() throws IOException { + this.client = new HttpHealthcareApiClient(); + } + + /** + * Execute Bundles. + * + * @param context the context + */ + @ProcessElement + public void executeBundles(ProcessContext context) { + String body = context.element(); + try { + // Validate that data was set to valid JSON. + mapper.readTree(body); + client.executeFhirBundle(fhirStore.get(), body); + } catch (IOException | HealthcareHttpException e) { + failedBundles.inc(); + context.output(HealthcareIOError.of(body, e)); + } + } + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java index 05f002ac2fc0..8167036bda56 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java @@ -27,6 +27,7 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; @@ -323,6 +324,8 @@ public void finishSpecifyingOutput( @Override public Result expand(PCollection input) { + CoderRegistry coderRegistry = input.getPipeline().getCoderRegistry(); + coderRegistry.registerCoderForClass(HL7v2Message.class, HL7v2MessageCoder.of()); return input.apply("Fetch HL7v2 messages", new FetchHL7v2Message()); } @@ -352,6 +355,8 @@ public FetchHL7v2Message() {} @Override public Result expand(PCollection msgIds) { + CoderRegistry coderRegistry = msgIds.getPipeline().getCoderRegistry(); + coderRegistry.registerCoderForClass(HL7v2Message.class, HL7v2MessageCoder.of()); return new Result( msgIds.apply( ParDo.of(new FetchHL7v2Message.HL7v2MessageGetFn()) @@ -470,6 +475,8 @@ public ListHL7v2Messages withInitialSplitDuration(Duration initialSplitDuration) @Override public PCollection expand(PBegin input) { + CoderRegistry coderRegistry = input.getPipeline().getCoderRegistry(); + coderRegistry.registerCoderForClass(HL7v2Message.class, HL7v2MessageCoder.of()); return input .apply(Create.ofProvider(this.hl7v2Stores, ListCoder.of(StringUtf8Coder.of()))) .apply(FlatMapElements.into(TypeDescriptors.strings()).via((x) -> x)) @@ -625,6 +632,8 @@ public abstract static class Write extends PTransform, @Override public Result expand(PCollection messages) { + CoderRegistry coderRegistry = messages.getPipeline().getCoderRegistry(); + coderRegistry.registerCoderForClass(HL7v2Message.class, HL7v2MessageCoder.of()); return messages.apply(new WriteHL7v2(this.getHL7v2Store(), this.getWriteMethod())); } @@ -728,7 +737,7 @@ public Write.Result expand(PCollection input) { PCollection> failedInserts = input .apply(ParDo.of(new WriteHL7v2Fn(hl7v2Store, writeMethod))) - .setCoder(new HealthcareIOErrorCoder<>(new HL7v2MessageCoder())); + .setCoder(HealthcareIOErrorCoder.of(HL7v2MessageCoder.of())); return Write.Result.in(input.getPipeline(), failedInserts); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2MessageCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2MessageCoder.java index a2402b8d5433..d1af961531ed 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2MessageCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2MessageCoder.java @@ -29,11 +29,15 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; public class HL7v2MessageCoder extends CustomCoder { + HL7v2MessageCoder() {} + public static HL7v2MessageCoder of() { return new HL7v2MessageCoder(); } - public HL7v2MessageCoder() {} + public static HL7v2MessageCoder of(Class clazz) { + return new HL7v2MessageCoder(); + } private static final NullableCoder STRING_CODER = NullableCoder.of(StringUtf8Coder.of()); private static final NullableCoder> MAP_CODER = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index afb57cd44d95..f504235a78b5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -18,15 +18,17 @@ package org.apache.beam.sdk.io.gcp.healthcare; import com.google.api.services.healthcare.v1beta1.model.Empty; +import com.google.api.services.healthcare.v1beta1.model.FhirStore; import com.google.api.services.healthcare.v1beta1.model.Hl7V2Store; import com.google.api.services.healthcare.v1beta1.model.HttpBody; import com.google.api.services.healthcare.v1beta1.model.IngestMessageResponse; import com.google.api.services.healthcare.v1beta1.model.ListMessagesResponse; import com.google.api.services.healthcare.v1beta1.model.Message; -import com.google.api.services.healthcare.v1beta1.model.SearchResourcesRequest; +import com.google.api.services.healthcare.v1beta1.model.Operation; import java.io.IOException; import java.text.ParseException; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.joda.time.Instant; /** Defines a client that talks to the Cloud Healthcare API. */ @@ -130,26 +132,11 @@ ListMessagesResponse makeHL7v2ListRequest( */ Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException; - /** - * Create fhir resource http body. - * - * @param fhirStore the fhir store - * @param type the type - * @param body the body - * @return the http body - * @throws IOException the io exception - */ - HttpBody createFhirResource(String fhirStore, String type, HttpBody body) throws IOException; + Operation importFhirResource( + String fhirStore, String gcsSourcePath, @Nullable String contentStructure) throws IOException; - /** - * Fhir search http body. - * - * @param fhirStore the fhir store - * @param query the query - * @return the http body - * @throws IOException the io exception - */ - HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException; + Operation pollOperation(Operation operation, Long sleepMs) + throws InterruptedException, IOException; /** * Execute fhir bundle http body. @@ -159,27 +146,17 @@ ListMessagesResponse makeHL7v2ListRequest( * @return the http body * @throws IOException the io exception */ - HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException; - - /** - * List fhir resource for patient http body. - * - * @param fhirStore the fhir store - * @param patient the patient - * @return the http body - * @throws IOException the io exception - */ - HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException; + HttpBody executeFhirBundle(String fhirStore, String bundle) + throws IOException, HealthcareHttpException; /** * Read fhir resource http body. * - * @param fhirStore the fhir store - * @param resource the resource + * @param resourceId the resource * @return the http body * @throws IOException the io exception */ - HttpBody readFHIRResource(String fhirStore, String resource) throws IOException; + HttpBody readFhirResource(String resourceId) throws IOException; /** * Create hl 7 v 2 store hl 7 v 2 store. @@ -191,6 +168,11 @@ ListMessagesResponse makeHL7v2ListRequest( */ Hl7V2Store createHL7v2Store(String dataset, String name) throws IOException; + FhirStore createFhirStore(String dataset, String name, String version, String pubsubTopic) + throws IOException; + + FhirStore createFhirStore(String dataset, String name, String version) throws IOException; + /** * Delete hl 7 v 2 store empty. * @@ -199,4 +181,6 @@ ListMessagesResponse makeHL7v2ListRequest( * @throws IOException the io exception */ Empty deleteHL7v2Store(String store) throws IOException; + + Empty deleteFhirStore(String store) throws IOException; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java index 97f94f8ccf5b..a97cde9ce1b1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java @@ -19,10 +19,13 @@ import com.google.api.client.googleapis.json.GoogleJsonResponseException; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; import org.joda.time.Instant; /** Class for capturing errors on IO operations on Google Cloud Healthcare APIs resources. */ +@DefaultCoder(HealthcareIOErrorCoder.class) public class HealthcareIOError { private T dataResource; private String errorMessage; @@ -39,7 +42,9 @@ public class HealthcareIOError { this.dataResource = dataResource; this.errorMessage = errorMessage; this.stackTrace = stackTrace; - this.statusCode = statusCode; + if (statusCode != null) { + this.statusCode = statusCode; + } if (observedTime != null) { this.observedTime = observedTime; } else { @@ -74,6 +79,8 @@ static HealthcareIOError of(T dataResource, Exception error) { if (error instanceof com.google.api.client.googleapis.json.GoogleJsonResponseException) { statusCode = ((GoogleJsonResponseException) error).getStatusCode(); + } else if (error instanceof HealthcareHttpException) { + statusCode = ((HealthcareHttpException) error).getStatusCode(); } return new HealthcareIOError<>(dataResource, msg, stackTrace, null, statusCode); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 46cb95b98790..653e9a855d68 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -28,17 +28,23 @@ import com.google.api.services.healthcare.v1beta1.CloudHealthcareScopes; import com.google.api.services.healthcare.v1beta1.model.CreateMessageRequest; import com.google.api.services.healthcare.v1beta1.model.Empty; +import com.google.api.services.healthcare.v1beta1.model.FhirStore; +import com.google.api.services.healthcare.v1beta1.model.GoogleCloudHealthcareV1beta1FhirRestGcsSource; import com.google.api.services.healthcare.v1beta1.model.Hl7V2Store; import com.google.api.services.healthcare.v1beta1.model.HttpBody; +import com.google.api.services.healthcare.v1beta1.model.ImportResourcesRequest; import com.google.api.services.healthcare.v1beta1.model.IngestMessageRequest; import com.google.api.services.healthcare.v1beta1.model.IngestMessageResponse; import com.google.api.services.healthcare.v1beta1.model.ListMessagesResponse; import com.google.api.services.healthcare.v1beta1.model.Message; -import com.google.api.services.healthcare.v1beta1.model.SearchResourcesRequest; +import com.google.api.services.healthcare.v1beta1.model.NotificationConfig; +import com.google.api.services.healthcare.v1beta1.model.Operation; +import com.google.api.services.storage.StorageScopes; import com.google.auth.oauth2.GoogleCredentials; import java.io.IOException; import java.io.Serializable; import java.net.URI; +import java.net.URISyntaxException; import java.text.ParseException; import java.util.ArrayList; import java.util.Iterator; @@ -48,7 +54,19 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.gcp.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.http.HttpEntity; +import org.apache.http.HttpResponse; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.methods.RequestBuilder; +import org.apache.http.client.utils.URIBuilder; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.DefaultHttpRequestRetryHandler; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,10 +76,19 @@ * mainly to encapsulate the unserializable dependencies, since most generated classes are not * serializable in the HTTP client. */ -public class HttpHealthcareApiClient implements HealthcareApiClient, Serializable { - - private transient CloudHealthcare client; +public class HttpHealthcareApiClient implements HealthcareApiClient, Serializable { + private static final String USER_AGENT = + String.format( + "apache-beam-io-google-cloud-platform-healthcare/%s", + ReleaseInfo.getReleaseInfo().getSdkVersion()); + private static final String FHIRSTORE_HEADER_CONTENT_TYPE = "application/fhir+json"; + private static final String FHIRSTORE_HEADER_ACCEPT = "application/fhir+json; charset=utf-8"; + private static final String FHIRSTORE_HEADER_ACCEPT_CHARSET = "utf-8"; private static final Logger LOG = LoggerFactory.getLogger(HttpHealthcareApiClient.class); + private transient CloudHealthcare client; + private transient HttpClient httpClient; + private transient GoogleCredentials credentials; + /** * Instantiates a new Http healthcare api client. * @@ -79,6 +106,7 @@ public HttpHealthcareApiClient() throws IOException { */ public HttpHealthcareApiClient(CloudHealthcare client) throws IOException { this.client = client; + this.httpClient = HttpClients.createDefault(); initClient(); } @@ -99,11 +127,45 @@ public Hl7V2Store createHL7v2Store(String dataset, String name) throws IOExcepti .execute(); } + @Override + public FhirStore createFhirStore(String dataset, String name, String version) throws IOException { + return createFhirStore(dataset, name, version, null); + } + + @Override + public FhirStore createFhirStore( + String dataset, String name, String version, @Nullable String pubsubTopic) + throws IOException { + FhirStore store = new FhirStore(); + + store.setVersion(version); + store.setDisableReferentialIntegrity(true); + store.setEnableUpdateCreate(true); + if (pubsubTopic != null) { + NotificationConfig notificationConfig = new NotificationConfig(); + notificationConfig.setPubsubTopic(pubsubTopic); + store.setNotificationConfig(notificationConfig); + } + return client + .projects() + .locations() + .datasets() + .fhirStores() + .create(dataset, store) + .setFhirStoreId(name) + .execute(); + } + @Override public Empty deleteHL7v2Store(String name) throws IOException { return client.projects().locations().datasets().hl7V2Stores().delete(name).execute(); } + @Override + public Empty deleteFhirStore(String name) throws IOException { + return client.projects().locations().datasets().fhirStores().delete(name).execute(); + } + @Override public Instant getEarliestHL7v2SendTime(String hl7v2Store, @Nullable String filter) throws IOException { @@ -288,18 +350,6 @@ public IngestMessageResponse ingestHL7v2Message(String hl7v2Store, Message msg) .execute(); } - @Override - public HttpBody fhirSearch(String fhirStore, SearchResourcesRequest query) throws IOException { - return client - .projects() - .locations() - .datasets() - .fhirStores() - .fhir() - .search(fhirStore, query) - .execute(); - } - @Override public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOException { CreateMessageRequest createMessageRequest = new CreateMessageRequest(); @@ -315,48 +365,120 @@ public Message createHL7v2Message(String hl7v2Store, Message msg) throws IOExcep } @Override - public HttpBody createFhirResource(String fhirStore, String type, HttpBody body) + public Operation importFhirResource( + String fhirStore, String gcsSourcePath, @Nullable String contentStructure) throws IOException { + GoogleCloudHealthcareV1beta1FhirRestGcsSource gcsSrc = + new GoogleCloudHealthcareV1beta1FhirRestGcsSource(); + + gcsSrc.setUri(gcsSourcePath); + ImportResourcesRequest importRequest = new ImportResourcesRequest(); + importRequest.setGcsSource(gcsSrc).setContentStructure(contentStructure); return client .projects() .locations() .datasets() .fhirStores() - .fhir() - .create(fhirStore, type, body) + .healthcareImport(fhirStore, importRequest) .execute(); } @Override - public HttpBody executeFhirBundle(String fhirStore, HttpBody bundle) throws IOException { - return client - .projects() - .locations() - .datasets() - .fhirStores() - .fhir() - .executeBundle(fhirStore, bundle) - .execute(); + public Operation pollOperation(Operation operation, Long sleepMs) + throws InterruptedException, IOException { + LOG.debug(String.format("started opertation %s. polling until complete.", operation.getName())); + while (operation.getDone() == null || !operation.getDone()) { + // Update the status of the operation with another request. + Thread.sleep(sleepMs); // Pause between requests. + operation = + client.projects().locations().datasets().operations().get(operation.getName()).execute(); + } + return operation; } @Override - public HttpBody listFHIRResourceForPatient(String fhirStore, String patient) throws IOException { - return client - .projects() - .locations() - .datasets() - .fhirStores() - .fhir() - .patientEverything(patient) - .execute(); + public HttpBody executeFhirBundle(String fhirStore, String bundle) + throws IOException, HealthcareHttpException { + if (httpClient == null || client == null) { + initClient(); + } + + credentials.refreshIfExpired(); + StringEntity requestEntity = new StringEntity(bundle, ContentType.APPLICATION_JSON); + URI uri; + try { + uri = + new URIBuilder(client.getRootUrl() + "v1beta1/" + fhirStore + "/fhir") + .setParameter("access_token", credentials.getAccessToken().getTokenValue()) + .build(); + } catch (URISyntaxException e) { + LOG.error("URL error when making executeBundle request to FHIR API. " + e.getMessage()); + throw new IllegalArgumentException(e); + } + + HttpUriRequest request = + RequestBuilder.post() + .setUri(uri) + .setEntity(requestEntity) + .addHeader("User-Agent", USER_AGENT) + .addHeader("Content-Type", FHIRSTORE_HEADER_CONTENT_TYPE) + .addHeader("Accept-Charset", FHIRSTORE_HEADER_ACCEPT_CHARSET) + .addHeader("Accept", FHIRSTORE_HEADER_ACCEPT) + .build(); + + HttpResponse response = httpClient.execute(request); + HttpEntity responseEntity = response.getEntity(); + String content = EntityUtils.toString(responseEntity); + // Check 2XX code. + if (!(response.getStatusLine().getStatusCode() / 100 == 2)) { + throw HealthcareHttpException.of(response); + } + HttpBody responseModel = new HttpBody(); + responseModel.setData(content); + return responseModel; + } + + /** + * Wraps {@link HttpResponse} in an exception with a statusCode field for use with {@link + * HealthcareIOError}. + */ + public static class HealthcareHttpException extends Exception { + private final Integer statusCode; + + HealthcareHttpException(HttpResponse response, String message) { + super(message); + this.statusCode = response.getStatusLine().getStatusCode(); + if (statusCode / 100 == 2) { + throw new IllegalArgumentException( + String.format( + "2xx codes should not be exceptions. Got status code: %s with body: %s", + statusCode, message)); + } + } + + /** + * Create Exception of {@link HttpResponse}. + * + * @param response the HTTP response + * @return the healthcare http exception + * @throws IOException the io exception + */ + static HealthcareHttpException of(HttpResponse response) throws IOException { + return new HealthcareHttpException(response, EntityUtils.toString(response.getEntity())); + } + + Integer getStatusCode() { + return statusCode; + } } @Override - public HttpBody readFHIRResource(String fhirStore, String resource) throws IOException { - return client.projects().locations().datasets().fhirStores().fhir().read(resource).execute(); + public HttpBody readFhirResource(String resourceId) throws IOException { + return client.projects().locations().datasets().fhirStores().fhir().read(resourceId).execute(); } - private static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer { + public static class AuthenticatedRetryInitializer extends RetryHttpRequestInitializer { + GoogleCredentials credentials; public AuthenticatedRetryInitializer(GoogleCredentials credentials) { @@ -367,11 +489,11 @@ public AuthenticatedRetryInitializer(GoogleCredentials credentials) { @Override public void initialize(HttpRequest request) throws IOException { super.initialize(request); + HttpHeaders requestHeaders = request.getHeaders(); + requestHeaders.setUserAgent(USER_AGENT); if (!credentials.hasRequestMetadata()) { return; } - HttpHeaders requestHeaders = request.getHeaders(); - requestHeaders.setUserAgent("apache-beam-hl7v2-io"); URI uri = null; if (request.getUrl() != null) { uri = request.getUrl().toURI(); @@ -389,19 +511,21 @@ public void initialize(HttpRequest request) throws IOException { } private void initClient() throws IOException { + + credentials = GoogleCredentials.getApplicationDefault(); // Create a HttpRequestInitializer, which will provide a baseline configuration to all requests. - // HttpRequestInitializer requestInitializer = new RetryHttpRequestInitializer(); - // GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); HttpRequestInitializer requestInitializer = new AuthenticatedRetryInitializer( - GoogleCredentials.getApplicationDefault() - .createScoped(CloudHealthcareScopes.CLOUD_PLATFORM)); + credentials.createScoped( + CloudHealthcareScopes.CLOUD_PLATFORM, StorageScopes.CLOUD_PLATFORM_READ_ONLY)); client = new CloudHealthcare.Builder( new NetHttpTransport(), new JacksonFactory(), requestInitializer) .setApplicationName("apache-beam-hl7v2-io") .build(); + httpClient = + HttpClients.custom().setRetryHandler(new DefaultHttpRequestRetryHandler(10, false)).build(); } public static class HL7v2MessagePages implements Iterable> { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java index 4ea0b7b62ab9..24b764f04af5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java @@ -162,7 +162,9 @@ static String createTopicName(Description description, String name) throws IOExc } if (description.getMethodName() != null) { - topicName.append(description.getMethodName()).append("-"); + // Remove braces (which are illegal in pubsub naming restrictions) in dynamic method names + // when using parameterized tests. + topicName.append(description.getMethodName().replaceAll("[\\[\\]]", "")).append("-"); } DATETIME_FORMAT.printTo(topicName, Instant.now()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java new file mode 100644 index 000000000000..71499bf98eb4 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java @@ -0,0 +1,147 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; + +import java.io.IOException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Collection; +import org.apache.beam.runners.direct.DirectOptions; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubGrpcClient; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.io.gcp.pubsub.TestPubsubOptions; +import org.apache.beam.sdk.io.gcp.pubsub.TestPubsubSignal; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class FhirIOReadIT { + + @Parameters(name = "{0}") + public static Collection versions() { + return Arrays.asList("DSTU2", "STU3", "R4"); + } + + @Rule public transient TestPubsubSignal signal = TestPubsubSignal.create(); + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + + private final String fhirStoreName; + private final String pubsubTopic; + private final String pubsubSubscription; + private final String project; + private transient HealthcareApiClient client; + private String healthcareDataset; + private PubsubClient pubsub; + private TestPubsubOptions pipelineOptions; + + public String version; + + public FhirIOReadIT(String version) { + this.version = version; + long testTime = System.currentTimeMillis(); + this.fhirStoreName = + "FHIR_store_" + version + "_write_it_" + testTime + "_" + (new SecureRandom().nextInt(32)); + this.project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + this.pubsubTopic = + "projects/" + + project + + "/topics/FhirIO-IT-" + + version + + "-notifications-" + + testTime + + "-" + + (new SecureRandom().nextInt(32)); + this.pubsubSubscription = pubsubTopic.replaceAll("topic", "subscription"); + pipelineOptions = TestPipeline.testingPipelineOptions().as(TestPubsubOptions.class); + } + + @Before + public void setup() throws Exception { + healthcareDataset = String.format(HEALTHCARE_DATASET_TEMPLATE, project); + if (client == null) { + this.client = new HttpHealthcareApiClient(); + } + pubsub = PubsubGrpcClient.FACTORY.newClient(null, null, pipelineOptions); + TopicPath topicPath = PubsubClient.topicPathFromPath(pubsubTopic); + pubsub.createTopic(topicPath); + SubscriptionPath subscriptionPath = PubsubClient.subscriptionPathFromPath(pubsubSubscription); + pubsub.createSubscription(topicPath, subscriptionPath, 60); + client.createFhirStore(healthcareDataset, fhirStoreName, version, pubsubTopic); + + // Execute bundles to trigger FHIR notificiations to input topic + FhirIOTestUtil.executeFhirBundles( + client, + healthcareDataset + "/fhirStores/" + fhirStoreName, + FhirIOTestUtil.BUNDLES.get(version)); + } + + @After + public void deleteFHIRtore() throws IOException { + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.deleteFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + TopicPath topicPath = PubsubClient.topicPathFromPath(pubsubTopic); + SubscriptionPath subscriptionPath = PubsubClient.subscriptionPathFromPath(pubsubSubscription); + pubsub.deleteSubscription(subscriptionPath); + pubsub.deleteTopic(topicPath); + pubsub.close(); + } + + @Test + public void testFhirIORead() throws Exception { + pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); + + FhirIO.Read.Result result = + pipeline + .apply(PubsubIO.readStrings().fromSubscription(pubsubSubscription)) + .apply(FhirIO.readResources()); + + PCollection resources = result.getResources(); + resources.apply( + "waitForAnyMessage", signal.signalSuccessWhen(resources.getCoder(), anyResources -> true)); + // wait for any resource + + Supplier start = signal.waitForStart(Duration.standardMinutes(5)); + pipeline.apply(signal.signalStart()); + PipelineResult job = pipeline.run(); + start.get(); + signal.waitForSuccess(Duration.standardSeconds(30)); + + // A runner may not support cancel + try { + job.cancel(); + } catch (UnsupportedOperationException exc) { + // noop + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java new file mode 100644 index 000000000000..976c55fcb76c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTest.java @@ -0,0 +1,112 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FhirIOTest { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + @Test + public void test_FhirIO_failedReads() { + List badMessageIDs = Arrays.asList("foo", "bar"); + FhirIO.Read.Result readResult = + pipeline.apply(Create.of(badMessageIDs)).apply(FhirIO.readResources()); + + PCollection> failed = readResult.getFailedReads(); + + PCollection resources = readResult.getResources(); + + PCollection failedMsgIds = + failed.apply( + MapElements.into(TypeDescriptors.strings()).via(HealthcareIOError::getDataResource)); + + PAssert.that(failedMsgIds).containsInAnyOrder(badMessageIDs); + PAssert.that(resources).empty(); + pipeline.run(); + } + + @Test + public void test_FhirIO_failedWrites() { + String badBundle = "bad"; + List emptyMessages = Collections.singletonList(badBundle); + + PCollection fhirBundles = pipeline.apply(Create.of(emptyMessages)); + + FhirIO.Write.Result writeResult = + fhirBundles.apply( + FhirIO.Write.executeBundles( + "projects/foo/locations/us-central1/datasets/bar/hl7V2Stores/baz")); + + PCollection> failedInserts = writeResult.getFailedBodies(); + + PAssert.thatSingleton(failedInserts) + .satisfies( + (HealthcareIOError err) -> { + Assert.assertEquals("bad", err.getDataResource()); + return null; + }); + PCollection numFailedInserts = failedInserts.apply(Count.globally()); + + PAssert.thatSingleton(numFailedInserts).isEqualTo(1L); + + pipeline.run(); + } + + private static final long NUM_ELEMENTS = 11; + private static final long BATCH_SIZE = 5; + + private static ArrayList> createTestData() { + String[] scientists = { + "Einstein", + "Darwin", + "Copernicus", + "Pasteur", + "Curie", + "Faraday", + "Newton", + "Bohr", + "Galilei", + "Maxwell" + }; + ArrayList> data = new ArrayList<>(); + for (int i = 0; i < NUM_ELEMENTS; i++) { + int index = i % scientists.length; + KV element = KV.of("key", scientists[index]); + data.add(element); + } + return data; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestOptions.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestOptions.java new file mode 100644 index 000000000000..33da786c0e4f --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestOptions.java @@ -0,0 +1,46 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation.Required; +import org.apache.beam.sdk.testing.TestPipelineOptions; + +public interface FhirIOTestOptions extends TestPipelineOptions { + + @Description( + "FHIR store should match the pattern: projects/PROJECT_ID/locations/LOCATION/datasets/DATASET_ID/fhirStores/HL7V2_STORE_ID") + @Required + String getFhirStore(); + + void setFhirStore(String value); + + @Description("GCS temp path for import should be of the form gs://bucket/path/") + @Default.String("gs://" + DEFAULT_TEMP_BUCKET + "/FhirIOWriteIT/temp/") + String getGcsTempPath(); + + void setGcsTempPath(String value); + + @Description("GCS dead letter path for import should be of the form gs://bucket/path/") + String getGcsDeadLetterPath(); + + void setGcsDeadLetterPath(String value); +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java new file mode 100644 index 000000000000..c893f59f744b --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -0,0 +1,127 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import com.google.api.client.http.HttpHeaders; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.client.http.javanet.NetHttpTransport; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.services.storage.Storage; +import com.google.api.services.storage.model.StorageObject; +import com.google.auth.oauth2.GoogleCredentials; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; + +class FhirIOTestUtil { + public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests"; + + private static Stream readPrettyBundles(String version) { + ClassLoader classLoader = FhirIOTestUtil.class.getClassLoader(); + Path resourceDir = Paths.get("build", "resources", "test", version); + String absolutePath = resourceDir.toFile().getAbsolutePath(); + File dir = new File(absolutePath); + File[] fhirJsons = dir.listFiles(); + return Arrays.stream(fhirJsons) + .map(File::toPath) + .map( + (Path path) -> { + try { + return Files.readAllBytes(path); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .map(String::new); + } + + // Could generate more messages at scale using a tool like + // https://synthetichealth.github.io/synthea/ if necessary chose not to avoid the dependency. + static final List DSTU2_PRETTY_BUNDLES = + readPrettyBundles("DSTU2").collect(Collectors.toList()); + static final List STU3_PRETTY_BUNDLES = + readPrettyBundles("STU3").collect(Collectors.toList()); + static final List R4_PRETTY_BUNDLES = + readPrettyBundles("R4").collect(Collectors.toList()); + + static final Map> BUNDLES; + + static { + Map> m = new HashMap<>(); + m.put("DSTU2", DSTU2_PRETTY_BUNDLES); + m.put("STU3", STU3_PRETTY_BUNDLES); + m.put("R4", R4_PRETTY_BUNDLES); + BUNDLES = Collections.unmodifiableMap(m); + } + + /** Populate the test resources into the FHIR store and returns a list of resource IDs. */ + static void executeFhirBundles(HealthcareApiClient client, String fhirStore, List bundles) + throws IOException, HealthcareHttpException { + for (String bundle : bundles) { + client.executeFhirBundle(fhirStore, bundle); + } + } + + public static void tearDownTempBucket() throws IOException { + + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + HttpRequestInitializer requestInitializer = + request -> { + HttpHeaders requestHeaders = request.getHeaders(); + if (!credentials.hasRequestMetadata()) { + return; + } + URI uri = null; + if (request.getUrl() != null) { + uri = request.getUrl().toURI(); + } + Map> credentialHeaders = credentials.getRequestMetadata(uri); + if (credentialHeaders == null) { + return; + } + for (Map.Entry> entry : credentialHeaders.entrySet()) { + String headerName = entry.getKey(); + List requestValues = new ArrayList<>(entry.getValue()); + requestHeaders.put(headerName, requestValues); + } + request.setConnectTimeout(60000); // 1 minute connect timeout + request.setReadTimeout(60000); // 1 minute read timeout + }; + Storage storage = + new Storage.Builder(new NetHttpTransport(), new JacksonFactory(), requestInitializer) + .build(); + List blobs = storage.objects().list(DEFAULT_TEMP_BUCKET).execute().getItems(); + if (blobs != null) { + for (StorageObject blob : blobs) { + storage.objects().delete(DEFAULT_TEMP_BUCKET, blob.getId()); + } + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java new file mode 100644 index 000000000000..24677ca3fb56 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java @@ -0,0 +1,129 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.BUNDLES; +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; +import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; + +import java.io.IOException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Collection; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class FhirIOWriteIT { + + @Parameters(name = "{0}") + public static Collection versions() { + return Arrays.asList("DSTU2", "STU3", "R4"); + } + + private final String fhirStoreName; + private FhirIOTestOptions options; + private transient HealthcareApiClient client; + private String healthcareDataset; + private long testTime = System.currentTimeMillis(); + + public String version; + + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + + public FhirIOWriteIT(String version) { + this.version = version; + this.fhirStoreName = + "FHIR_store_" + version + "_write_it_" + testTime + "_" + (new SecureRandom().nextInt(32)); + } + + @Before + public void setup() throws Exception { + if (client == null) { + client = new HttpHealthcareApiClient(); + } + PipelineOptionsFactory.register(FhirIOTestOptions.class); + String project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + healthcareDataset = String.format(HEALTHCARE_DATASET_TEMPLATE, project); + options = TestPipeline.testingPipelineOptions().as(FhirIOTestOptions.class); + options.setGcsTempPath( + String.format("gs://%s/FhirIOWrite%sIT/%s/temp/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setGcsDeadLetterPath( + String.format( + "gs://%s/FhirIOWrite%sIT/%s/deadletter/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.createFhirStore(healthcareDataset, fhirStoreName, version); + } + + @After + public void teardownFhirStore() throws IOException { + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.deleteFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + // clean up GCS objects if any. + } + + @AfterClass + public static void teardownBucket() throws IOException { + FhirIOTestUtil.tearDownTempBucket(); + } + + @Test + public void testFhirIO_ExecuteBundle() throws IOException { + FhirIO.Write.Result writeResult = + pipeline + .apply(Create.of(BUNDLES.get(version))) + .apply(FhirIO.Write.executeBundles(options.getFhirStore())); + + PAssert.that(writeResult.getFailedBodies()).empty(); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void testFhirIO_Import() { + Pipeline pipeline = Pipeline.create(options); + options.setTempLocation("gs://temp-storage-for-healthcare-io-tests"); + FhirIO.Write.Result result = + pipeline + .apply(Create.of(BUNDLES.get(version))) + .apply( + FhirIO.Write.fhirStoresImport( + options.getFhirStore(), + options.getGcsDeadLetterPath(), + ContentStructure.BUNDLE)); + + PAssert.that(result.getFailedBodies()).empty(); + PAssert.that(result.getFailedFiles()).empty(); + + pipeline.run().waitUntilFinish(); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java index 94f6ed6086ea..d3769b8fa3db 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java @@ -92,7 +92,7 @@ public void testHL7v2IO_ListHL7v2Messages() throws Exception { PCollection result = pipeline.apply(HL7v2IO.read(healthcareDataset + "/hl7V2Stores/" + HL7V2_STORE_NAME)); PCollection numReadMessages = - result.setCoder(new HL7v2MessageCoder()).apply(Count.globally()); + result.setCoder(HL7v2MessageCoder.of()).apply(Count.globally()); PAssert.thatSingleton(numReadMessages).isEqualTo((long) MESSAGES.size()); PAssert.that(result) @@ -119,7 +119,7 @@ public void testHL7v2IO_ListHL7v2Messages_filtered() throws Exception { HL7v2IO.readWithFilter( healthcareDataset + "/hl7V2Stores/" + HL7V2_STORE_NAME, adtFilter)); PCollection numReadMessages = - result.setCoder(new HL7v2MessageCoder()).apply(Count.globally()); + result.setCoder(HL7v2MessageCoder.of()).apply(Count.globally()); PAssert.thatSingleton(numReadMessages).isEqualTo(NUM_ADT); PAssert.that(result) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java index eaffc536ca8b..8a4566bffc26 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java @@ -101,7 +101,7 @@ public void testHL7v2IOE2E() throws Exception { .apply(HL7v2IO.getAll()); PCollection numReadMessages = - readResult.getMessages().setCoder(new HL7v2MessageCoder()).apply(Count.globally()); + readResult.getMessages().setCoder(HL7v2MessageCoder.of()).apply(Count.globally()); PAssert.thatSingleton(numReadMessages).isEqualTo((long) MESSAGES.size()); PAssert.that(readResult.getFailedReads()).empty(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java index f52d60744367..88a09e539c21 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java @@ -82,7 +82,7 @@ public void tearDown() throws Exception { public void testHL7v2IOWrite() throws Exception { HL7v2IO.Write.Result result = pipeline - .apply(Create.of(MESSAGES).withCoder(new HL7v2MessageCoder())) + .apply(Create.of(MESSAGES).withCoder(HL7v2MessageCoder.of())) .apply(HL7v2IO.ingestMessages(healthcareDataset + "/hl7V2Stores/" + HL7V2_STORE_NAME)); PAssert.that(result.getFailedInsertsWithErr()).empty(); diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json new file mode 100644 index 000000000000..677f4a31ef9e --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json @@ -0,0 +1,14018 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3", + "resource": { + "resourceType": "Patient", + "id": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 3529060201889641636 Population seed: 1586309754086
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Race", + "code": "2106-3", + "display": "White" + } + ], + "text": "White" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Ethnicity", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + ], + "text": "Not Hispanic or Latino" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Shantay950 Collier206" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Quincy", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 2.756674276028057 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 63.24332572397194 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR" + } + ] + }, + "system": "http://hospital.smarthealthit.org", + "value": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB" + } + ] + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-91-5467" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "DL" + } + ] + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99971576" + } + ], + "name": [ + { + "use": "official", + "family": [ + "Krajcik437" + ], + "given": [ + "Amelia635" + ], + "prefix": [ + "Mrs." + ] + }, + { + "use": "maiden", + "family": [ + "Hoeger474" + ], + "given": [ + "Amelia635" + ], + "prefix": [ + "Mrs." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-617-5450", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1953-09-25", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 41.56777541317302 + }, + { + "url": "longitude", + "valueDecimal": -70.99411465843632 + } + ] + } + ], + "line": [ + "1056 Stark Trace Suite 42" + ], + "city": "Dartmouth", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "M" + } + ] + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c", + "resource": { + "resourceType": "Organization", + "id": "f9914571-32b8-36d3-a422-4cdce9fbb86c", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "INDEPENDENCE EYE ASSOCIATES, PC", + "telecom": [ + { + "system": "phone", + "value": "508-985-6600" + } + ], + "address": [ + { + "line": [ + "365 FAUNCE CORNER RD" + ], + "city": "DARTMOUTH", + "state": "MA", + "postalCode": "02747-6230", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b", + "resource": { + "resourceType": "Practitioner", + "id": "7e727355-643f-3e2b-b89f-f5076660354b", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "61860" + } + ], + "active": true, + "name": { + "family": [ + "Salazar800" + ], + "given": [ + "Rodrigo242" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "365 FAUNCE CORNER RD" + ], + "city": "DARTMOUTH", + "state": "MA", + "postalCode": "02747-6230", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55", + "resource": { + "resourceType": "Encounter", + "id": "0f99c02d-1199-4d87-8459-123025910c55", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "1993-10-01T19:52:39-07:00", + "end": "1993-10-01T20:07:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3eaf24a3-0c67-4505-a02e-fcc09a4093a3", + "resource": { + "resourceType": "Condition", + "id": "3eaf24a3-0c67-4505-a02e-fcc09a4093a3", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55" + }, + "dateRecorded": "1993-10-01", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "1993-10-01T19:52:39-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:0319b3fc-3373-42f6-9b08-6ee695e640b0", + "resource": { + "resourceType": "CarePlan", + "id": "0319b3fc-3373-42f6-9b08-6ee695e640b0", + "text": { + "status": "generated", + "div": "
    Diabetes self management plan
    " + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "active", + "context": { + "reference": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55" + }, + "period": { + "start": "1993-10-01T19:52:39-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698360004", + "display": "Diabetes self management plan" + } + ], + "text": "Diabetes self management plan" + } + ], + "addresses": [ + { + "reference": "urn:uuid:3eaf24a3-0c67-4505-a02e-fcc09a4093a3" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "160670007", + "display": "Diabetic diet" + } + ], + "text": "Diabetic diet" + }, + "status": "in-progress", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:fa8baaad-a3ec-4e4a-9cd3-483bb9283cdb", + "resource": { + "resourceType": "Claim", + "id": "fa8baaad-a3ec-4e4a-9cd3-483bb9283cdb", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92", + "resource": { + "resourceType": "Organization", + "id": "ecc51621-0af3-3b35-ac3e-8b1e34022e92", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "SAINT ANNE'S HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5086745600" + } + ], + "address": [ + { + "line": [ + "795 MIDDLE STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02721", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d", + "resource": { + "resourceType": "Practitioner", + "id": "38a05f68-cc3b-33b6-993e-1d2bb2696c3d", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "100" + } + ], + "active": true, + "name": { + "family": [ + "Padberg411" + ], + "given": [ + "Ligia986" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "795 MIDDLE STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02721", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:29a7628c-9d6d-400a-a495-c6714208d248", + "resource": { + "resourceType": "Encounter", + "id": "29a7628c-9d6d-400a-a495-c6714208d248", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "1997-11-21T18:52:39-08:00", + "end": "1997-11-21T19:37:39-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7e78b2fa-5f76-4547-b107-93e9bff9a3fd", + "resource": { + "resourceType": "Condition", + "id": "7e78b2fa-5f76-4547-b107-93e9bff9a3fd", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:29a7628c-9d6d-400a-a495-c6714208d248" + }, + "dateRecorded": "1997-11-21", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + ], + "text": "Miscarriage in first trimester" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "1997-11-21T18:52:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f56d5dc9-3ad3-4b81-9dd3-6b9adcdafeb0", + "resource": { + "resourceType": "Claim", + "id": "f56d5dc9-3ad3-4b81-9dd3-6b9adcdafeb0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315", + "resource": { + "resourceType": "Encounter", + "id": "ed060d5a-328c-4c0f-a88e-8bfb00655315", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2008-09-11T19:52:39-07:00", + "end": "2008-09-11T20:07:39-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "239872002", + "display": "Osteoarthritis of hip" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394", + "resource": { + "resourceType": "Condition", + "id": "b5bfd448-96ad-4bdd-b3be-816b0d018394", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" + }, + "dateRecorded": "2008-09-11", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "239872002", + "display": "Osteoarthritis of hip" + } + ], + "text": "Osteoarthritis of hip" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2008-09-11T19:52:39-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:46597927-f43d-43c3-9482-50dcc695c22a", + "resource": { + "resourceType": "MedicationOrder", + "id": "46597927-f43d-43c3-9482-50dcc695c22a", + "dateWritten": "2008-09-11T19:52:39-07:00", + "status": "active", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "prescriber": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + }, + "encounter": { + "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" + }, + "reasonReference": { + "reference": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "849574", + "display": "Naproxen sodium 220 MG Oral Tablet" + } + ], + "text": "Naproxen sodium 220 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:a237e458-a9ef-4914-88c4-7faed80b0730", + "resource": { + "resourceType": "Claim", + "id": "a237e458-a9ef-4914-88c4-7faed80b0730", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:46597927-f43d-43c3-9482-50dcc695c22a" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6fbc1d80-0ee3-48ae-8efc-e404a9720e5e", + "resource": { + "resourceType": "CarePlan", + "id": "6fbc1d80-0ee3-48ae-8efc-e404a9720e5e", + "text": { + "status": "generated", + "div": "
    Musculoskeletal care
    " + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "active", + "context": { + "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" + }, + "period": { + "start": "2008-09-11T19:52:39-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "408869004", + "display": "Musculoskeletal care" + } + ], + "text": "Musculoskeletal care" + } + ], + "addresses": [ + { + "reference": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "386294003", + "display": "Joint mobility exercises" + } + ], + "text": "Joint mobility exercises" + }, + "status": "in-progress", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266694003", + "display": "Heat therapy" + } + ], + "text": "Heat therapy" + }, + "status": "in-progress", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:04cac450-c4b6-41ad-8b78-e3dff5734386", + "resource": { + "resourceType": "Claim", + "id": "04cac450-c4b6-41ad-8b78-e3dff5734386", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "239872002", + "display": "Osteoarthritis of hip" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d", + "resource": { + "resourceType": "Organization", + "id": "f7a6cc79-3075-3124-9f62-6fe562440b1d", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f7a6cc79-3075-3124-9f62-6fe562440b1d" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "HAWTHORN MEDICAL URGENT CARE CENTER", + "telecom": [ + { + "system": "phone", + "value": "508-961-0861" + } + ], + "address": [ + { + "line": [ + "237 STATE ROAD" + ], + "city": "NORTH DARTMOUTH", + "state": "MA", + "postalCode": "2747", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a", + "resource": { + "resourceType": "Practitioner", + "id": "d4b02c0c-6cc4-31dd-bc8a-817f786b043a", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93530" + } + ], + "active": true, + "name": { + "family": [ + "Doyle959" + ], + "given": [ + "Willis868" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "237 STATE ROAD" + ], + "city": "NORTH DARTMOUTH", + "state": "MA", + "postalCode": "2747", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f", + "resource": { + "resourceType": "Encounter", + "id": "659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a" + } + } + ], + "period": { + "start": "2010-05-14T19:52:39-07:00", + "end": "2010-05-14T20:07:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6bd6be0c-2cd0-493d-af74-32311a4a7563", + "resource": { + "resourceType": "Immunization", + "id": "6bd6be0c-2cd0-493d-af74-32311a4a7563", + "status": "completed", + "date": "2010-05-14T19:52:39-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:641b5155-f8b5-4845-b015-80d81aa480bc", + "resource": { + "resourceType": "Claim", + "id": "641b5155-f8b5-4845-b015-80d81aa480bc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de", + "resource": { + "resourceType": "Encounter", + "id": "ea9e3c13-3ad3-40df-9cdd-f3cd817944de", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2010-05-08T19:52:39-07:00", + "end": "2010-05-08T20:19:39-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1", + "resource": { + "resourceType": "Condition", + "id": "6f19df56-d88d-41d3-8d1a-a940a50e19e1", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" + }, + "dateRecorded": "2010-05-08", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-05-08T19:52:39-07:00", + "abatementDateTime": "2010-05-15T19:52:39-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f1b6cbda-e1fc-4547-88e4-0a8b9699a3bf", + "resource": { + "resourceType": "Procedure", + "id": "f1b6cbda-e1fc-4547-88e4-0a8b9699a3bf", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "reasonReference": { + "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" + }, + "performedPeriod": { + "start": "2010-05-08T19:52:39-07:00", + "end": "2010-05-08T20:04:39-07:00" + }, + "encounter": { + "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4353ffd8-d9e2-4538-ae2f-d026939a7431", + "resource": { + "resourceType": "MedicationOrder", + "id": "4353ffd8-d9e2-4538-ae2f-d026939a7431", + "dateWritten": "2010-05-08T19:52:39-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "prescriber": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + }, + "encounter": { + "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" + }, + "reasonReference": { + "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:ddf48aaa-8260-44b2-9519-883d0f3a18c7", + "resource": { + "resourceType": "Claim", + "id": "ddf48aaa-8260-44b2-9519-883d0f3a18c7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:4353ffd8-d9e2-4538-ae2f-d026939a7431" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d7cc9124-e4c3-4076-9a11-f72545d524ac", + "resource": { + "resourceType": "CarePlan", + "id": "d7cc9124-e4c3-4076-9a11-f72545d524ac", + "text": { + "status": "generated", + "div": "
    Respiratory therapy
    " + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "context": { + "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" + }, + "period": { + "start": "2010-05-08T19:52:39-07:00", + "end": "2010-11-05T19:52:39-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "addresses": [ + { + "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:c179e666-caf6-41f6-9871-a1f148f3fb45", + "resource": { + "resourceType": "Claim", + "id": "c179e666-caf6-41f6-9871-a1f148f3fb45", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac", + "resource": { + "resourceType": "Encounter", + "id": "da87062c-3694-46e9-94a5-c09ca489b8ac", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2010-11-05T19:52:39-07:00", + "end": "2010-11-05T20:07:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8b1b3a3d-807f-463a-8d61-e282a732d0a8", + "resource": { + "resourceType": "Observation", + "id": "8b1b3a3d-807f-463a-8d61-e282a732d0a8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:43e738a5-a338-42d2-b130-76a484437e42", + "resource": { + "resourceType": "Observation", + "id": "43e738a5-a338-42d2-b130-76a484437e42", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:08fa42ee-365e-477e-b8ba-0c81ceb3154a", + "resource": { + "resourceType": "Observation", + "id": "08fa42ee-365e-477e-b8ba-0c81ceb3154a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2a38c5e-12c5-41d7-8109-a220f426492f", + "resource": { + "resourceType": "Observation", + "id": "a2a38c5e-12c5-41d7-8109-a220f426492f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f7fa66f-5509-4a78-b907-781587a39c8f", + "resource": { + "resourceType": "Observation", + "id": "9f7fa66f-5509-4a78-b907-781587a39c8f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:435f220c-64d2-4b34-aeae-80832cbbd085", + "resource": { + "resourceType": "Observation", + "id": "435f220c-64d2-4b34-aeae-80832cbbd085", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b249e0c-445c-4234-a15c-a59b08fc4f8f", + "resource": { + "resourceType": "Observation", + "id": "2b249e0c-445c-4234-a15c-a59b08fc4f8f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b89ebbea-7f9a-48f8-a9aa-167d53862888", + "resource": { + "resourceType": "Observation", + "id": "b89ebbea-7f9a-48f8-a9aa-167d53862888", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 80.020, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:52920ed8-3af5-4773-a3ac-6a7d4e7af012", + "resource": { + "resourceType": "Observation", + "id": "52920ed8-3af5-4773-a3ac-6a7d4e7af012", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 16.280, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d2cee4f-acb7-490b-a871-5e4d69c9dfb3", + "resource": { + "resourceType": "Observation", + "id": "3d2cee4f-acb7-490b-a871-5e4d69c9dfb3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 0.86000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53f78b14-7682-4a7b-9aae-8ec9cbcfd21c", + "resource": { + "resourceType": "Observation", + "id": "53f78b14-7682-4a7b-9aae-8ec9cbcfd21c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 9.9000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8998aa3-70fd-401c-bf72-d123b7953747", + "resource": { + "resourceType": "Observation", + "id": "e8998aa3-70fd-401c-bf72-d123b7953747", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 138.78, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8cc2fb6-7a8d-4229-b286-d7d8b2f8d98c", + "resource": { + "resourceType": "Observation", + "id": "b8cc2fb6-7a8d-4229-b286-d7d8b2f8d98c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 3.9800, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:02d268e7-fb6b-457a-a3ab-1bdf7389eb89", + "resource": { + "resourceType": "Observation", + "id": "02d268e7-fb6b-457a-a3ab-1bdf7389eb89", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 103.58, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e073447-f476-493f-9d37-179a11f58b31", + "resource": { + "resourceType": "Observation", + "id": "6e073447-f476-493f-9d37-179a11f58b31", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 25.740, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7fc8f159-8ffa-41a5-aa58-26520c7ecab1", + "resource": { + "resourceType": "Observation", + "id": "7fc8f159-8ffa-41a5-aa58-26520c7ecab1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 9.0808, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70121fba-da3f-4a1e-b612-f20a3cb6d042", + "resource": { + "resourceType": "Observation", + "id": "70121fba-da3f-4a1e-b612-f20a3cb6d042", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 4.9309, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:21e182da-dff6-441e-b0c2-f3a3b9b46f62", + "resource": { + "resourceType": "Observation", + "id": "21e182da-dff6-441e-b0c2-f3a3b9b46f62", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 16.191, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a84065c7-e904-4acc-ad51-9f39668f5695", + "resource": { + "resourceType": "Observation", + "id": "a84065c7-e904-4acc-ad51-9f39668f5695", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 45.164, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7276f2fb-37f4-4901-b075-53e748e67265", + "resource": { + "resourceType": "Observation", + "id": "7276f2fb-37f4-4901-b075-53e748e67265", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 80.755, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:001d3228-0bcf-43ba-af90-7d1c0fb22190", + "resource": { + "resourceType": "Observation", + "id": "001d3228-0bcf-43ba-af90-7d1c0fb22190", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 32.529, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4ad7fa9-a31c-421f-ac6c-032ecfa80f88", + "resource": { + "resourceType": "Observation", + "id": "d4ad7fa9-a31c-421f-ac6c-032ecfa80f88", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 33.172, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ff9d720-bd87-4a2b-b7e2-1182635b0c09", + "resource": { + "resourceType": "Observation", + "id": "1ff9d720-bd87-4a2b-b7e2-1182635b0c09", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 39.778, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29421a10-66db-4a10-b7fc-6e99bad82ef3", + "resource": { + "resourceType": "Observation", + "id": "29421a10-66db-4a10-b7fc-6e99bad82ef3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 200.68, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee462f37-2bc1-494a-9706-330cdb17267d", + "resource": { + "resourceType": "Observation", + "id": "ee462f37-2bc1-494a-9706-330cdb17267d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 360.68, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:411f3285-ed40-4b43-9288-565c8353f19b", + "resource": { + "resourceType": "Observation", + "id": "411f3285-ed40-4b43-9288-565c8353f19b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 11.053, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:97aedff2-37bb-4e9b-8ba2-441480c02f1a", + "resource": { + "resourceType": "Observation", + "id": "97aedff2-37bb-4e9b-8ba2-441480c02f1a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fe9119d8-fe40-4773-bf67-208dc746e5cc", + "resource": { + "resourceType": "Observation", + "id": "fe9119d8-fe40-4773-bf67-208dc746e5cc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "valueQuantity": { + "value": 6.3200, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:989350b4-5feb-4567-a916-41f6e36179dd", + "resource": { + "resourceType": "DiagnosticReport", + "id": "989350b4-5feb-4567-a916-41f6e36179dd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:6e073447-f476-493f-9d37-179a11f58b31", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:4522a9d2-fc57-49a3-9561-4e478e48f37c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4522a9d2-fc57-49a3-9561-4e478e48f37c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" + }, + "effectiveDateTime": "2010-11-05T19:52:39-07:00", + "issued": "2010-11-05T19:52:39.493-07:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:411f3285-ed40-4b43-9288-565c8353f19b", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:ca2d559e-05ed-43ea-8d51-3d281198d2be", + "resource": { + "resourceType": "Claim", + "id": "ca2d559e-05ed-43ea-8d51-3d281198d2be", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9", + "resource": { + "resourceType": "Encounter", + "id": "ab4d28a2-a2dd-48a0-b4ea-0184705382b9", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2011-11-11T18:52:39-08:00", + "end": "2011-11-11T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3dfedde6-6c6c-4ce8-91d4-a69c97cd29b1", + "resource": { + "resourceType": "Observation", + "id": "3dfedde6-6c6c-4ce8-91d4-a69c97cd29b1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:091b9bb5-514a-4b0a-b22e-ad8c8cfb3197", + "resource": { + "resourceType": "Observation", + "id": "091b9bb5-514a-4b0a-b22e-ad8c8cfb3197", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a315c78-e0fe-41d6-bd62-475f38830ad4", + "resource": { + "resourceType": "Observation", + "id": "3a315c78-e0fe-41d6-bd62-475f38830ad4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7ea95d9-0113-4f45-be07-cac688820b87", + "resource": { + "resourceType": "Observation", + "id": "a7ea95d9-0113-4f45-be07-cac688820b87", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71e8b565-18c8-46d4-ad26-32da02e3ddbb", + "resource": { + "resourceType": "Observation", + "id": "71e8b565-18c8-46d4-ad26-32da02e3ddbb", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 131, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7fd3111c-08ce-416e-be77-c5b2980272d0", + "resource": { + "resourceType": "Observation", + "id": "7fd3111c-08ce-416e-be77-c5b2980272d0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 67, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c19d44e0-1f51-4536-bd8b-8b048b59725f", + "resource": { + "resourceType": "Observation", + "id": "c19d44e0-1f51-4536-bd8b-8b048b59725f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7f3898da-13ec-487a-ab99-dbe319b2a766", + "resource": { + "resourceType": "Observation", + "id": "7f3898da-13ec-487a-ab99-dbe319b2a766", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 95.5, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:658cf76a-f850-4861-ad91-7ce546b57af9", + "resource": { + "resourceType": "Observation", + "id": "658cf76a-f850-4861-ad91-7ce546b57af9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 12.450, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:615f8813-b216-43d8-8d41-a818bdd2273a", + "resource": { + "resourceType": "Observation", + "id": "615f8813-b216-43d8-8d41-a818bdd2273a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.80000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:575089a7-49ca-4597-9320-2ac878aa9f9a", + "resource": { + "resourceType": "Observation", + "id": "575089a7-49ca-4597-9320-2ac878aa9f9a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.5400, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8e567f2-47ab-4159-ba36-e7b3e07834f4", + "resource": { + "resourceType": "Observation", + "id": "d8e567f2-47ab-4159-ba36-e7b3e07834f4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 141.62, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:222e6976-7948-4231-af06-de764fd8c6b8", + "resource": { + "resourceType": "Observation", + "id": "222e6976-7948-4231-af06-de764fd8c6b8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.3100, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d3fca14-4f95-43dd-a886-c13b5f046096", + "resource": { + "resourceType": "Observation", + "id": "6d3fca14-4f95-43dd-a886-c13b5f046096", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 102.16, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c3ac129a-5767-42f2-8605-01516a1cb7e0", + "resource": { + "resourceType": "Observation", + "id": "c3ac129a-5767-42f2-8605-01516a1cb7e0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 21.610, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dcc3146f-9369-47f4-ba13-6f92bbbffa98", + "resource": { + "resourceType": "Observation", + "id": "dcc3146f-9369-47f4-ba13-6f92bbbffa98", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9aaadfd6-2c24-4442-bf5d-20452e200f7a", + "resource": { + "resourceType": "Observation", + "id": "9aaadfd6-2c24-4442-bf5d-20452e200f7a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "valueQuantity": { + "value": 6, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c6227e1-508f-481c-b374-40c62af3f271", + "resource": { + "resourceType": "Immunization", + "id": "3c6227e1-508f-481c-b374-40c62af3f271", + "status": "completed", + "date": "2011-11-11T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:6a0919b3-c980-40ae-9ba5-96e4ad156c4d", + "resource": { + "resourceType": "DiagnosticReport", + "id": "6a0919b3-c980-40ae-9ba5-96e4ad156c4d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" + }, + "effectiveDateTime": "2011-11-11T18:52:39-08:00", + "issued": "2011-11-11T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:c3ac129a-5767-42f2-8605-01516a1cb7e0", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b2dcf71d-3448-45f1-9445-e6bae561611b", + "resource": { + "resourceType": "Claim", + "id": "b2dcf71d-3448-45f1-9445-e6bae561611b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486", + "resource": { + "resourceType": "Encounter", + "id": "670309a3-be34-417a-8d78-13b8d1ca2486", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2012-11-16T18:52:39-08:00", + "end": "2012-11-16T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4f2afec2-c2b6-47d5-a05d-aaca9787eef6", + "resource": { + "resourceType": "Observation", + "id": "4f2afec2-c2b6-47d5-a05d-aaca9787eef6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ba4ee99-1274-4aea-a766-e82c74267faf", + "resource": { + "resourceType": "Observation", + "id": "2ba4ee99-1274-4aea-a766-e82c74267faf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:226b2507-471b-41da-9021-0210f0de940e", + "resource": { + "resourceType": "Observation", + "id": "226b2507-471b-41da-9021-0210f0de940e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6933e334-8d1c-4121-87d9-63e263a8b641", + "resource": { + "resourceType": "Observation", + "id": "6933e334-8d1c-4121-87d9-63e263a8b641", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70afdfbf-f71f-425e-bc08-ce46996c8557", + "resource": { + "resourceType": "Observation", + "id": "70afdfbf-f71f-425e-bc08-ce46996c8557", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 101, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bcd9105a-7d96-4af8-8795-645a0e6a8493", + "resource": { + "resourceType": "Observation", + "id": "bcd9105a-7d96-4af8-8795-645a0e6a8493", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 85, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:787dbac8-ad30-45ab-a1af-e96f19c435ae", + "resource": { + "resourceType": "Observation", + "id": "787dbac8-ad30-45ab-a1af-e96f19c435ae", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df7f409c-d231-422d-b039-73dd61efbe75", + "resource": { + "resourceType": "Observation", + "id": "df7f409c-d231-422d-b039-73dd61efbe75", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 75.810, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3d2924a-e4da-4b42-9d77-baba07de69a6", + "resource": { + "resourceType": "Observation", + "id": "f3d2924a-e4da-4b42-9d77-baba07de69a6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 8.8300, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a9dcf68-fdd2-41d4-a3da-cec3111093e4", + "resource": { + "resourceType": "Observation", + "id": "3a9dcf68-fdd2-41d4-a3da-cec3111093e4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.79000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b37615d-d752-42d8-9ecd-5592ecddfe21", + "resource": { + "resourceType": "Observation", + "id": "8b37615d-d752-42d8-9ecd-5592ecddfe21", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.7900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9727c9a7-bcf0-496d-9cde-11a9b0f6221d", + "resource": { + "resourceType": "Observation", + "id": "9727c9a7-bcf0-496d-9cde-11a9b0f6221d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 139.39, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2083699-11a0-483a-ba2f-5a6c958af14b", + "resource": { + "resourceType": "Observation", + "id": "b2083699-11a0-483a-ba2f-5a6c958af14b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.9200, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:404a813a-e432-4250-a06c-e7718ddcc4be", + "resource": { + "resourceType": "Observation", + "id": "404a813a-e432-4250-a06c-e7718ddcc4be", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 105, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3916a2b-1805-4264-84c1-11c1662bb295", + "resource": { + "resourceType": "Observation", + "id": "b3916a2b-1805-4264-84c1-11c1662bb295", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 27.990, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ce62e32-1557-49f7-a397-5f2e78379659", + "resource": { + "resourceType": "Observation", + "id": "6ce62e32-1557-49f7-a397-5f2e78379659", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 164.71, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82929ea8-e275-4c33-b62e-483ecd08d995", + "resource": { + "resourceType": "Observation", + "id": "82929ea8-e275-4c33-b62e-483ecd08d995", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 102.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8abd3258-d1ad-40a8-99ab-fd8deb1e9844", + "resource": { + "resourceType": "Observation", + "id": "8abd3258-d1ad-40a8-99ab-fd8deb1e9844", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 75.650, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3da90c1a-0a99-4b7c-9710-8a38c8d97e45", + "resource": { + "resourceType": "Observation", + "id": "3da90c1a-0a99-4b7c-9710-8a38c8d97e45", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 68.510, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:082e903a-1466-4702-b881-9e0465ed88ea", + "resource": { + "resourceType": "Observation", + "id": "082e903a-1466-4702-b881-9e0465ed88ea", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f270ddac-c2b0-45bd-82c9-6c6ea4462fb6", + "resource": { + "resourceType": "Observation", + "id": "f270ddac-c2b0-45bd-82c9-6c6ea4462fb6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "valueQuantity": { + "value": 6.2800, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d128e8a-5a6d-47ff-8a83-8fa7f98c9cb1", + "resource": { + "resourceType": "Immunization", + "id": "9d128e8a-5a6d-47ff-8a83-8fa7f98c9cb1", + "status": "completed", + "date": "2012-11-16T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:65d5da8c-bb67-470b-b070-65f85e921dfc", + "resource": { + "resourceType": "DiagnosticReport", + "id": "65d5da8c-bb67-470b-b070-65f85e921dfc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:b3916a2b-1805-4264-84c1-11c1662bb295", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:d1b936cf-567e-4591-93ad-6fb29c2478fa", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d1b936cf-567e-4591-93ad-6fb29c2478fa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" + }, + "effectiveDateTime": "2012-11-16T18:52:39-08:00", + "issued": "2012-11-16T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:3da90c1a-0a99-4b7c-9710-8a38c8d97e45", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:ff296cf4-dc5a-4dee-b556-055cb4d59325", + "resource": { + "resourceType": "Claim", + "id": "ff296cf4-dc5a-4dee-b556-055cb4d59325", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788", + "resource": { + "resourceType": "Encounter", + "id": "011ec61e-e99b-4ffe-a5c3-b6f28020f788", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2013-11-22T18:52:39-08:00", + "end": "2013-11-22T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:af0c6c9d-ed83-4a58-9258-bfb6223e715b", + "resource": { + "resourceType": "Observation", + "id": "af0c6c9d-ed83-4a58-9258-bfb6223e715b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8c1cc80d-217d-413e-a182-cbcf73ced89b", + "resource": { + "resourceType": "Observation", + "id": "8c1cc80d-217d-413e-a182-cbcf73ced89b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:672c6bd6-fbfa-476d-b955-7763b0ca5b51", + "resource": { + "resourceType": "Observation", + "id": "672c6bd6-fbfa-476d-b955-7763b0ca5b51", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:63a2d450-7607-422b-81d8-e01dc856db60", + "resource": { + "resourceType": "Observation", + "id": "63a2d450-7607-422b-81d8-e01dc856db60", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a999933d-7a0b-4129-b92c-b47715840c10", + "resource": { + "resourceType": "Observation", + "id": "a999933d-7a0b-4129-b92c-b47715840c10", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7f8b06c1-cf34-4473-9879-30edfb70e009", + "resource": { + "resourceType": "Observation", + "id": "7f8b06c1-cf34-4473-9879-30edfb70e009", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 68, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c3c2dc96-a3af-46a5-ab19-88027cf20680", + "resource": { + "resourceType": "Observation", + "id": "c3c2dc96-a3af-46a5-ab19-88027cf20680", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9a6113db-31c6-43be-b20b-a81b92ff4891", + "resource": { + "resourceType": "Observation", + "id": "9a6113db-31c6-43be-b20b-a81b92ff4891", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 71.610, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfd84a6d-9c18-4040-8179-16cc385cac6e", + "resource": { + "resourceType": "Observation", + "id": "cfd84a6d-9c18-4040-8179-16cc385cac6e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 10.160, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e835fffe-4304-4967-9aea-506bfb3b83d8", + "resource": { + "resourceType": "Observation", + "id": "e835fffe-4304-4967-9aea-506bfb3b83d8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 1.0200, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5632b342-89c2-40dd-a00a-9bd866d2f503", + "resource": { + "resourceType": "Observation", + "id": "5632b342-89c2-40dd-a00a-9bd866d2f503", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.3000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0bc0e869-58d3-4ad4-abdf-0a96cec68053", + "resource": { + "resourceType": "Observation", + "id": "0bc0e869-58d3-4ad4-abdf-0a96cec68053", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 141.72, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:54e932c8-ea30-4135-9b5c-1b4dfdfaa131", + "resource": { + "resourceType": "Observation", + "id": "54e932c8-ea30-4135-9b5c-1b4dfdfaa131", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.1600, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfb92202-11de-4434-92b3-103728dfae5e", + "resource": { + "resourceType": "Observation", + "id": "cfb92202-11de-4434-92b3-103728dfae5e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 101.73, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c5de72f9-db83-4f60-adfb-a4d1df56bd86", + "resource": { + "resourceType": "Observation", + "id": "c5de72f9-db83-4f60-adfb-a4d1df56bd86", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 22.640, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8dba8b3-11f2-45ec-a5d9-9ef30d22dc18", + "resource": { + "resourceType": "Observation", + "id": "b8dba8b3-11f2-45ec-a5d9-9ef30d22dc18", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:07b11457-1de7-41fb-a472-47de4dd42564", + "resource": { + "resourceType": "Observation", + "id": "07b11457-1de7-41fb-a472-47de4dd42564", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "valueQuantity": { + "value": 6.1600, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40283994-a00b-41a4-b910-9e08dc302acf", + "resource": { + "resourceType": "Immunization", + "id": "40283994-a00b-41a4-b910-9e08dc302acf", + "status": "completed", + "date": "2013-11-22T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:595336ba-d00d-4061-a4e4-7f288a552add", + "resource": { + "resourceType": "DiagnosticReport", + "id": "595336ba-d00d-4061-a4e4-7f288a552add", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" + }, + "effectiveDateTime": "2013-11-22T18:52:39-08:00", + "issued": "2013-11-22T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:c5de72f9-db83-4f60-adfb-a4d1df56bd86", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b2714247-19be-41c2-b878-91bd9bf24e52", + "resource": { + "resourceType": "Claim", + "id": "b2714247-19be-41c2-b878-91bd9bf24e52", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313", + "resource": { + "resourceType": "Encounter", + "id": "950f6970-4fe6-4205-bba3-7be621435313", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2014-03-17T19:52:39-07:00", + "end": "2014-03-17T20:07:39-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fbc36a68-14be-4eba-9c54-09c0968f9dbc", + "resource": { + "resourceType": "Condition", + "id": "fbc36a68-14be-4eba-9c54-09c0968f9dbc", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313" + }, + "dateRecorded": "2014-03-17", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2014-03-17T19:52:39-07:00", + "abatementDateTime": "2014-03-28T19:52:39-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:4d70c627-0881-4c1d-b1d3-a0f1e49d1566", + "resource": { + "resourceType": "Observation", + "id": "4d70c627-0881-4c1d-b1d3-a0f1e49d1566", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313" + }, + "effectiveDateTime": "2014-03-17T19:52:39-07:00", + "issued": "2014-03-17T19:52:39.493-07:00", + "valueQuantity": { + "value": 37.283, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:58d48e5e-86e6-4598-99f7-4bb6fbda952e", + "resource": { + "resourceType": "Claim", + "id": "58d48e5e-86e6-4598-99f7-4bb6fbda952e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189", + "resource": { + "resourceType": "Encounter", + "id": "1823d974-5838-40b6-acc2-a1674c659189", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2014-11-28T18:52:39-08:00", + "end": "2014-11-28T19:22:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1f3be8cd-4779-477d-b3e5-8301376b3c2d", + "resource": { + "resourceType": "Observation", + "id": "1f3be8cd-4779-477d-b3e5-8301376b3c2d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47467834-5b7d-45bf-bdb5-7f9fed78242a", + "resource": { + "resourceType": "Observation", + "id": "47467834-5b7d-45bf-bdb5-7f9fed78242a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:976cd9e8-bfd6-4222-aa8c-3756da9ababe", + "resource": { + "resourceType": "Observation", + "id": "976cd9e8-bfd6-4222-aa8c-3756da9ababe", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:42911871-bd0b-484a-a603-fb5da39d120c", + "resource": { + "resourceType": "Observation", + "id": "42911871-bd0b-484a-a603-fb5da39d120c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f8f32e7-006b-487a-aa6f-539cfa9f5203", + "resource": { + "resourceType": "Observation", + "id": "5f8f32e7-006b-487a-aa6f-539cfa9f5203", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6cf975b-7aa0-4f85-bf7f-b502abdeadbe", + "resource": { + "resourceType": "Observation", + "id": "b6cf975b-7aa0-4f85-bf7f-b502abdeadbe", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53507506-838b-433c-8916-7e0f4e6279ee", + "resource": { + "resourceType": "Observation", + "id": "53507506-838b-433c-8916-7e0f4e6279ee", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2ff4d3e-015f-4668-a207-48aaf1916d47", + "resource": { + "resourceType": "Observation", + "id": "b2ff4d3e-015f-4668-a207-48aaf1916d47", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 88.860, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4459a77-32fd-468f-8658-623c067748c4", + "resource": { + "resourceType": "Observation", + "id": "d4459a77-32fd-468f-8658-623c067748c4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 10.870, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:349b712c-d8ad-4ac8-8b2c-937af8c37274", + "resource": { + "resourceType": "Observation", + "id": "349b712c-d8ad-4ac8-8b2c-937af8c37274", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.97000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ed99f79-1036-4c8b-adc9-620519d809ba", + "resource": { + "resourceType": "Observation", + "id": "6ed99f79-1036-4c8b-adc9-620519d809ba", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.4400, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47355ffc-38d7-4027-af0a-885bad94b53b", + "resource": { + "resourceType": "Observation", + "id": "47355ffc-38d7-4027-af0a-885bad94b53b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 137.21, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ae3afdcb-7a3e-4d12-bb74-7b59bbe6dfd3", + "resource": { + "resourceType": "Observation", + "id": "ae3afdcb-7a3e-4d12-bb74-7b59bbe6dfd3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.6100, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50c7e3ef-8731-4795-a3df-37b151388a09", + "resource": { + "resourceType": "Observation", + "id": "50c7e3ef-8731-4795-a3df-37b151388a09", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 108.45, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", + "resource": { + "resourceType": "Observation", + "id": "0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 22.850, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82c756b4-052a-4f7f-989e-8b8b508d3ccf", + "resource": { + "resourceType": "Observation", + "id": "82c756b4-052a-4f7f-989e-8b8b508d3ccf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3aa43a9b-1fca-4e34-9a14-c48892499715", + "resource": { + "resourceType": "Observation", + "id": "3aa43a9b-1fca-4e34-9a14-c48892499715", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "valueQuantity": { + "value": 6.0600, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a38b30bc-b795-4f4f-adf6-02700db3b80c", + "resource": { + "resourceType": "Procedure", + "id": "a38b30bc-b795-4f4f-adf6-02700db3b80c", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2014-11-28T18:52:39-08:00", + "end": "2014-11-28T19:07:39-08:00" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:53ac6448-eb28-458e-b036-087ae17f31bf", + "resource": { + "resourceType": "Immunization", + "id": "53ac6448-eb28-458e-b036-087ae17f31bf", + "status": "completed", + "date": "2014-11-28T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c8a80d5d-f63d-4419-9104-ee22c6d867b9", + "resource": { + "resourceType": "Immunization", + "id": "c8a80d5d-f63d-4419-9104-ee22c6d867b9", + "status": "completed", + "date": "2014-11-28T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:14ece09e-a0e7-4dd1-8738-3e513ef89bf4", + "resource": { + "resourceType": "DiagnosticReport", + "id": "14ece09e-a0e7-4dd1-8738-3e513ef89bf4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" + }, + "effectiveDateTime": "2014-11-28T18:52:39-08:00", + "issued": "2014-11-28T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0a38c1c9-20ab-458c-82c1-f8e77b071720", + "resource": { + "resourceType": "Claim", + "id": "0a38c1c9-20ab-458c-82c1-f8e77b071720", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 430.76, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c", + "resource": { + "resourceType": "Encounter", + "id": "d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2015-01-18T18:52:39-08:00", + "end": "2015-01-18T19:37:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b1e119c2-1643-4099-92ce-9e95d8942b2b", + "resource": { + "resourceType": "Procedure", + "id": "b1e119c2-1643-4099-92ce-9e95d8942b2b", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "performedPeriod": { + "start": "2015-01-18T18:52:39-08:00", + "end": "2015-01-18T19:22:39-08:00" + }, + "encounter": { + "reference": "urn:uuid:d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:db0f1238-1920-44a6-98ec-5a83648c1ed8", + "resource": { + "resourceType": "Claim", + "id": "db0f1238-1920-44a6-98ec-5a83648c1ed8", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + }, + "net": { + "value": 21085.78, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847", + "resource": { + "resourceType": "Encounter", + "id": "01bfb335-6313-4898-8ab8-0e461e5bb847", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2015-12-04T18:52:39-08:00", + "end": "2015-12-04T19:22:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:99323959-640c-4e5c-96fe-c2e6bd099513", + "resource": { + "resourceType": "Observation", + "id": "99323959-640c-4e5c-96fe-c2e6bd099513", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c2e9798f-b55f-4e1f-adba-1bec89e3f699", + "resource": { + "resourceType": "Observation", + "id": "c2e9798f-b55f-4e1f-adba-1bec89e3f699", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f471de46-c10a-47ee-942e-57765eea248a", + "resource": { + "resourceType": "Observation", + "id": "f471de46-c10a-47ee-942e-57765eea248a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca903c50-c4ab-47f8-8244-088405cf5977", + "resource": { + "resourceType": "Observation", + "id": "ca903c50-c4ab-47f8-8244-088405cf5977", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b32a170e-8ad0-4101-99c0-dd13459da450", + "resource": { + "resourceType": "Observation", + "id": "b32a170e-8ad0-4101-99c0-dd13459da450", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e13dda94-5e79-4666-9719-8d8bcc23a158", + "resource": { + "resourceType": "Observation", + "id": "e13dda94-5e79-4666-9719-8d8bcc23a158", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e80a2fd3-e367-4211-8132-3b8ca1d8cd22", + "resource": { + "resourceType": "Observation", + "id": "e80a2fd3-e367-4211-8132-3b8ca1d8cd22", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ad4d891-dc2d-42d2-bfc9-2bca5eb3a05e", + "resource": { + "resourceType": "Observation", + "id": "5ad4d891-dc2d-42d2-bfc9-2bca5eb3a05e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 70.200, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c26e5b59-394d-4662-a62e-77ef3f9d96ab", + "resource": { + "resourceType": "Observation", + "id": "c26e5b59-394d-4662-a62e-77ef3f9d96ab", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 18.720, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddca3175-5a43-49b9-bfde-c4425efa81dc", + "resource": { + "resourceType": "Observation", + "id": "ddca3175-5a43-49b9-bfde-c4425efa81dc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.76000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4165197d-501f-4753-9c91-5a46c07fb0fa", + "resource": { + "resourceType": "Observation", + "id": "4165197d-501f-4753-9c91-5a46c07fb0fa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 8.5700, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2be499cc-d580-4d23-b32c-87159a09f7a0", + "resource": { + "resourceType": "Observation", + "id": "2be499cc-d580-4d23-b32c-87159a09f7a0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 141.15, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67ea8ba3-0169-4499-8c32-1e2ef07e3565", + "resource": { + "resourceType": "Observation", + "id": "67ea8ba3-0169-4499-8c32-1e2ef07e3565", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.0100, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47e1ed26-65c3-47b9-b907-232b2c667189", + "resource": { + "resourceType": "Observation", + "id": "47e1ed26-65c3-47b9-b907-232b2c667189", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 104.60, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:802da8b4-db40-4b00-9c4b-35d3a0056cfe", + "resource": { + "resourceType": "Observation", + "id": "802da8b4-db40-4b00-9c4b-35d3a0056cfe", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 23.390, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e78b7d8-3b20-4b02-9946-f7187d3d0090", + "resource": { + "resourceType": "Observation", + "id": "7e78b7d8-3b20-4b02-9946-f7187d3d0090", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 195.21, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:651d9a01-8dfa-4c78-a49d-b6b4c4a7a85c", + "resource": { + "resourceType": "Observation", + "id": "651d9a01-8dfa-4c78-a49d-b6b4c4a7a85c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 111.03, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbe2464b-b19e-4c6d-8425-a5f6a20399c1", + "resource": { + "resourceType": "Observation", + "id": "bbe2464b-b19e-4c6d-8425-a5f6a20399c1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 106.03, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9681baf6-4b51-4144-965c-9c96314821ae", + "resource": { + "resourceType": "Observation", + "id": "9681baf6-4b51-4144-965c-9c96314821ae", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 66.970, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:356edd60-4206-4e34-b6e2-1ce68bd8c693", + "resource": { + "resourceType": "Observation", + "id": "356edd60-4206-4e34-b6e2-1ce68bd8c693", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 3.8256, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa94d2bf-a78c-4718-b8cb-163dfe7ce830", + "resource": { + "resourceType": "Observation", + "id": "fa94d2bf-a78c-4718-b8cb-163dfe7ce830", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.1438, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb83beb7-9084-49e4-8cc0-6c83a8f7ef27", + "resource": { + "resourceType": "Observation", + "id": "cb83beb7-9084-49e4-8cc0-6c83a8f7ef27", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 17.086, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7234c947-45c7-4ba3-adae-6a5a0dda91f2", + "resource": { + "resourceType": "Observation", + "id": "7234c947-45c7-4ba3-adae-6a5a0dda91f2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 40.758, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d14adbcb-bd94-4985-9878-76b461cd5516", + "resource": { + "resourceType": "Observation", + "id": "d14adbcb-bd94-4985-9878-76b461cd5516", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 84.708, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f0205de1-9351-42fe-901e-c7ab23fb24a1", + "resource": { + "resourceType": "Observation", + "id": "f0205de1-9351-42fe-901e-c7ab23fb24a1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 27.499, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5b801394-0d88-48d6-9864-696d3fb6a22a", + "resource": { + "resourceType": "Observation", + "id": "5b801394-0d88-48d6-9864-696d3fb6a22a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 34.677, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8fca8466-132c-4377-ab19-03557741684b", + "resource": { + "resourceType": "Observation", + "id": "8fca8466-132c-4377-ab19-03557741684b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 40.780, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:796a0b69-d993-4936-b606-62b4ff53d7f7", + "resource": { + "resourceType": "Observation", + "id": "796a0b69-d993-4936-b606-62b4ff53d7f7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 266.53, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8ea379cd-681a-43c1-a72e-46599a656369", + "resource": { + "resourceType": "Observation", + "id": "8ea379cd-681a-43c1-a72e-46599a656369", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 253.01, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8f5bb42a-e434-40c5-bfb8-3407a226cc0a", + "resource": { + "resourceType": "Observation", + "id": "8f5bb42a-e434-40c5-bfb8-3407a226cc0a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 10.838, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aea84217-58b6-4cb5-b1b3-ead39313a8fc", + "resource": { + "resourceType": "Observation", + "id": "aea84217-58b6-4cb5-b1b3-ead39313a8fc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ef2f503-3ef7-488c-8a57-cd4f82a9f85b", + "resource": { + "resourceType": "Observation", + "id": "9ef2f503-3ef7-488c-8a57-cd4f82a9f85b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "valueQuantity": { + "value": 5.9300, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a8518237-a0ef-4ebf-9735-cc01f0ddc276", + "resource": { + "resourceType": "Procedure", + "id": "a8518237-a0ef-4ebf-9735-cc01f0ddc276", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2015-12-04T18:52:39-08:00", + "end": "2015-12-04T19:07:39-08:00" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f74727ba-81ed-4ad4-b6a5-764ed865a6fc", + "resource": { + "resourceType": "Immunization", + "id": "f74727ba-81ed-4ad4-b6a5-764ed865a6fc", + "status": "completed", + "date": "2015-12-04T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:513034fd-ba93-400c-9ecc-698b450c3ff1", + "resource": { + "resourceType": "DiagnosticReport", + "id": "513034fd-ba93-400c-9ecc-698b450c3ff1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:802da8b4-db40-4b00-9c4b-35d3a0056cfe", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:854138ea-366e-40a6-9a4d-6b6bde7425f1", + "resource": { + "resourceType": "DiagnosticReport", + "id": "854138ea-366e-40a6-9a4d-6b6bde7425f1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:9681baf6-4b51-4144-965c-9c96314821ae", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:30438cdb-e42c-4b8b-a101-2a85ee93f30c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "30438cdb-e42c-4b8b-a101-2a85ee93f30c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" + }, + "effectiveDateTime": "2015-12-04T18:52:39-08:00", + "issued": "2015-12-04T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:8f5bb42a-e434-40c5-bfb8-3407a226cc0a", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:982a470b-06d6-472d-8642-16f022877d46", + "resource": { + "resourceType": "Claim", + "id": "982a470b-06d6-472d-8642-16f022877d46", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 502.94, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a", + "resource": { + "resourceType": "Encounter", + "id": "37444168-0363-4600-ab6b-b4e0088e768a", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2016-12-09T18:52:39-08:00", + "end": "2016-12-09T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6e39bfdc-b174-47be-bbf5-5df749a688d5", + "resource": { + "resourceType": "Observation", + "id": "6e39bfdc-b174-47be-bbf5-5df749a688d5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0300e321-3e6b-42a4-b9f7-2d88e7e1f064", + "resource": { + "resourceType": "Observation", + "id": "0300e321-3e6b-42a4-b9f7-2d88e7e1f064", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66773dbc-f8f7-4aa2-87de-0b254a619292", + "resource": { + "resourceType": "Observation", + "id": "66773dbc-f8f7-4aa2-87de-0b254a619292", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:85d4484b-1080-4a9c-b719-ae0980a75b47", + "resource": { + "resourceType": "Observation", + "id": "85d4484b-1080-4a9c-b719-ae0980a75b47", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b84ef9ef-d30b-4e2e-b85a-14668e26b597", + "resource": { + "resourceType": "Observation", + "id": "b84ef9ef-d30b-4e2e-b85a-14668e26b597", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 87, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ea15b6f4-f751-4627-b18f-f910e36e8426", + "resource": { + "resourceType": "Observation", + "id": "ea15b6f4-f751-4627-b18f-f910e36e8426", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 92, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:141de97b-1235-47b4-99fc-15ddb63e75e6", + "resource": { + "resourceType": "Observation", + "id": "141de97b-1235-47b4-99fc-15ddb63e75e6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:705eefa9-baf2-4245-8292-36c5c4e6349b", + "resource": { + "resourceType": "Observation", + "id": "705eefa9-baf2-4245-8292-36c5c4e6349b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 81.430, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9558c234-0562-48f0-92e4-c596c2b6a90e", + "resource": { + "resourceType": "Observation", + "id": "9558c234-0562-48f0-92e4-c596c2b6a90e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 14.080, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35cb1a99-71a0-47be-8e23-1bd70a79dff7", + "resource": { + "resourceType": "Observation", + "id": "35cb1a99-71a0-47be-8e23-1bd70a79dff7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 1.0700, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2647f5fd-e871-4d3b-bb29-3b7e41701208", + "resource": { + "resourceType": "Observation", + "id": "2647f5fd-e871-4d3b-bb29-3b7e41701208", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 8.6900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:585c9a48-d551-45a3-92b5-7009c801d000", + "resource": { + "resourceType": "Observation", + "id": "585c9a48-d551-45a3-92b5-7009c801d000", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 141.85, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a6e89c79-e52e-473f-bd3e-5f0e30571590", + "resource": { + "resourceType": "Observation", + "id": "a6e89c79-e52e-473f-bd3e-5f0e30571590", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.2300, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:348e3f9e-5841-429e-9c62-0b1cec36e6d5", + "resource": { + "resourceType": "Observation", + "id": "348e3f9e-5841-429e-9c62-0b1cec36e6d5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 106.70, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", + "resource": { + "resourceType": "Observation", + "id": "1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 24.660, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd46f602-9656-454d-8423-70b63e1f5b97", + "resource": { + "resourceType": "Observation", + "id": "dd46f602-9656-454d-8423-70b63e1f5b97", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:efd5875c-ece5-4724-89fa-434121fe45cb", + "resource": { + "resourceType": "Observation", + "id": "efd5875c-ece5-4724-89fa-434121fe45cb", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "valueQuantity": { + "value": 5.8500, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf27f8d0-6820-41d5-8878-01471bdb4b43", + "resource": { + "resourceType": "Immunization", + "id": "cf27f8d0-6820-41d5-8878-01471bdb4b43", + "status": "completed", + "date": "2016-12-09T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:72440146-8e9b-4b58-ae8f-9533f3265e2a", + "resource": { + "resourceType": "DiagnosticReport", + "id": "72440146-8e9b-4b58-ae8f-9533f3265e2a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" + }, + "effectiveDateTime": "2016-12-09T18:52:39-08:00", + "issued": "2016-12-09T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:307ff1d2-2101-4760-beb8-385b9a1a5acb", + "resource": { + "resourceType": "Claim", + "id": "307ff1d2-2101-4760-beb8-385b9a1a5acb", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356", + "resource": { + "resourceType": "Encounter", + "id": "19d53601-0ca8-4316-9435-40265d2f9356", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2017-12-15T18:52:39-08:00", + "end": "2017-12-15T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b40a71dd-1cb6-4c88-a626-176437e4f810", + "resource": { + "resourceType": "Observation", + "id": "b40a71dd-1cb6-4c88-a626-176437e4f810", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:732bb789-7c07-4bdf-805d-f6cd44a0a62a", + "resource": { + "resourceType": "Observation", + "id": "732bb789-7c07-4bdf-805d-f6cd44a0a62a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ce32e9ed-59c5-4799-9e64-e37d0d4fc188", + "resource": { + "resourceType": "Observation", + "id": "ce32e9ed-59c5-4799-9e64-e37d0d4fc188", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:10e191a6-f2fe-4c8b-9c3e-ce145ba0c8b9", + "resource": { + "resourceType": "Observation", + "id": "10e191a6-f2fe-4c8b-9c3e-ce145ba0c8b9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ea326672-4519-4534-8645-0272fcd3f1df", + "resource": { + "resourceType": "Observation", + "id": "ea326672-4519-4534-8645-0272fcd3f1df", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 118, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aea79c24-32ce-4a4b-a18a-4a94f3187a87", + "resource": { + "resourceType": "Observation", + "id": "aea79c24-32ce-4a4b-a18a-4a94f3187a87", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:13b57867-9f2b-472b-9e27-8a7e300c14c4", + "resource": { + "resourceType": "Observation", + "id": "13b57867-9f2b-472b-9e27-8a7e300c14c4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5dc175ad-a4ce-412c-ad98-dca0a3aecda1", + "resource": { + "resourceType": "Observation", + "id": "5dc175ad-a4ce-412c-ad98-dca0a3aecda1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 75.900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99e923b3-ffe9-4dd6-9871-22eabbf599d1", + "resource": { + "resourceType": "Observation", + "id": "99e923b3-ffe9-4dd6-9871-22eabbf599d1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 17.090, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fbabe4df-6b2e-414d-8b79-97e1dc491f5f", + "resource": { + "resourceType": "Observation", + "id": "fbabe4df-6b2e-414d-8b79-97e1dc491f5f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.74000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:86aa7ee4-407f-43d0-9fdf-571858dd7ebc", + "resource": { + "resourceType": "Observation", + "id": "86aa7ee4-407f-43d0-9fdf-571858dd7ebc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.5300, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:be50d438-9336-44ec-9528-2a437941c1c7", + "resource": { + "resourceType": "Observation", + "id": "be50d438-9336-44ec-9528-2a437941c1c7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 139.55, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9be8fdb-d8ed-4a4c-a9c3-9ae1c4cb1317", + "resource": { + "resourceType": "Observation", + "id": "a9be8fdb-d8ed-4a4c-a9c3-9ae1c4cb1317", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.3300, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7ce122f8-9814-41ba-b23f-241f47fa0ca7", + "resource": { + "resourceType": "Observation", + "id": "7ce122f8-9814-41ba-b23f-241f47fa0ca7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 108.81, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32619173-042b-4fae-8c5e-8577e1583f19", + "resource": { + "resourceType": "Observation", + "id": "32619173-042b-4fae-8c5e-8577e1583f19", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 21.960, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ada54f9-ba2a-49ab-9fff-1c4c6dfe43d4", + "resource": { + "resourceType": "Observation", + "id": "5ada54f9-ba2a-49ab-9fff-1c4c6dfe43d4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d625ed0e-6efd-430d-a858-b604a027f7a8", + "resource": { + "resourceType": "Observation", + "id": "d625ed0e-6efd-430d-a858-b604a027f7a8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "valueQuantity": { + "value": 6.3300, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1275a54a-8321-44fc-84c7-8eeb39188b31", + "resource": { + "resourceType": "Immunization", + "id": "1275a54a-8321-44fc-84c7-8eeb39188b31", + "status": "completed", + "date": "2017-12-15T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7a20af2f-3f74-4af0-8a81-df0e2dfddd17", + "resource": { + "resourceType": "DiagnosticReport", + "id": "7a20af2f-3f74-4af0-8a81-df0e2dfddd17", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" + }, + "effectiveDateTime": "2017-12-15T18:52:39-08:00", + "issued": "2017-12-15T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:32619173-042b-4fae-8c5e-8577e1583f19", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b9ab9708-2f0e-4ba1-a64c-324bddcdfc72", + "resource": { + "resourceType": "Claim", + "id": "b9ab9708-2f0e-4ba1-a64c-324bddcdfc72", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8e088420-73c4-4220-8908-20e829ab4cdf", + "resource": { + "resourceType": "Encounter", + "id": "8e088420-73c4-4220-8908-20e829ab4cdf", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2018-04-28T19:52:39-07:00", + "end": "2018-04-28T20:07:39-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0f4f91b8-4b10-4f5f-b94c-aebb27dd4adf", + "resource": { + "resourceType": "Condition", + "id": "0f4f91b8-4b10-4f5f-b94c-aebb27dd4adf", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:8e088420-73c4-4220-8908-20e829ab4cdf" + }, + "dateRecorded": "2018-04-28", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2018-04-28T19:52:39-07:00", + "abatementDateTime": "2018-05-19T19:52:39-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:a63c2763-4797-47e1-bdd6-286ded36b473", + "resource": { + "resourceType": "Claim", + "id": "a63c2763-4797-47e1-bdd6-286ded36b473", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f", + "resource": { + "resourceType": "Encounter", + "id": "14d585fa-87b6-48e6-b152-2e7c7cf42e8f", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2018-12-21T18:52:39-08:00", + "end": "2018-12-21T19:07:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f0da06c3-4e82-4bad-b0ce-8cdd3a35fed5", + "resource": { + "resourceType": "Observation", + "id": "f0da06c3-4e82-4bad-b0ce-8cdd3a35fed5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8036198b-2419-48ba-9379-d3aad37b83b5", + "resource": { + "resourceType": "Observation", + "id": "8036198b-2419-48ba-9379-d3aad37b83b5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:424ac9cd-a043-4d18-b98d-7ada684410a3", + "resource": { + "resourceType": "Observation", + "id": "424ac9cd-a043-4d18-b98d-7ada684410a3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 82.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ab45213-4f0c-4ac3-aa3f-edf5a8609243", + "resource": { + "resourceType": "Observation", + "id": "9ab45213-4f0c-4ac3-aa3f-edf5a8609243", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9eefb9cf-7abf-4bba-be76-cda789699973", + "resource": { + "resourceType": "Observation", + "id": "9eefb9cf-7abf-4bba-be76-cda789699973", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 88, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c50c7b2b-2dc1-4257-8b73-5ddfc7e431c8", + "resource": { + "resourceType": "Observation", + "id": "c50c7b2b-2dc1-4257-8b73-5ddfc7e431c8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 77, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d368f28e-dd42-48b1-b627-655459ca34b0", + "resource": { + "resourceType": "Observation", + "id": "d368f28e-dd42-48b1-b627-655459ca34b0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:286c0b51-4e55-4f09-8d88-5e477f08a78e", + "resource": { + "resourceType": "Observation", + "id": "286c0b51-4e55-4f09-8d88-5e477f08a78e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 74.830, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c3f782c5-1b4b-4cad-988b-3281a6c3e458", + "resource": { + "resourceType": "Observation", + "id": "c3f782c5-1b4b-4cad-988b-3281a6c3e458", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 10.580, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:597cc2e1-27e3-43f7-a4d1-7fd6ed2f1e59", + "resource": { + "resourceType": "Observation", + "id": "597cc2e1-27e3-43f7-a4d1-7fd6ed2f1e59", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 0.73000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e7b0f59c-0e17-4ada-bc9f-24f795399999", + "resource": { + "resourceType": "Observation", + "id": "e7b0f59c-0e17-4ada-bc9f-24f795399999", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 8.7000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d669964f-aad9-4d32-ac0e-24f939ee1bea", + "resource": { + "resourceType": "Observation", + "id": "d669964f-aad9-4d32-ac0e-24f939ee1bea", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 141.81, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a67392a6-a9c2-43ed-980d-000c20ce2bb8", + "resource": { + "resourceType": "Observation", + "id": "a67392a6-a9c2-43ed-980d-000c20ce2bb8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 4.8000, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d19c5563-b910-4f11-a909-dcfff2fa0328", + "resource": { + "resourceType": "Observation", + "id": "d19c5563-b910-4f11-a909-dcfff2fa0328", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 104.07, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", + "resource": { + "resourceType": "Observation", + "id": "85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 25.740, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5414a56b-7478-4e39-800f-42addf185b68", + "resource": { + "resourceType": "Observation", + "id": "5414a56b-7478-4e39-800f-42addf185b68", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 176.96, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2f58954-2a4c-4545-b908-42cd1f29eed6", + "resource": { + "resourceType": "Observation", + "id": "a2f58954-2a4c-4545-b908-42cd1f29eed6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 115.38, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:492c726d-55f5-4387-b3b7-91e7de7c9456", + "resource": { + "resourceType": "Observation", + "id": "492c726d-55f5-4387-b3b7-91e7de7c9456", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 84.670, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:abc0ddcb-9d31-4fec-b134-319d4aca0760", + "resource": { + "resourceType": "Observation", + "id": "abc0ddcb-9d31-4fec-b134-319d4aca0760", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 69.210, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d40a0cd-355d-4295-9767-c856c2a3671e", + "resource": { + "resourceType": "Observation", + "id": "6d40a0cd-355d-4295-9767-c856c2a3671e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8fb75f2e-6374-4b3b-9b84-8e0080e1a865", + "resource": { + "resourceType": "Observation", + "id": "8fb75f2e-6374-4b3b-9b84-8e0080e1a865", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "valueQuantity": { + "value": 6.2200, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f2046a66-34a7-49fa-955b-8766220f9c33", + "resource": { + "resourceType": "Immunization", + "id": "f2046a66-34a7-49fa-955b-8766220f9c33", + "status": "completed", + "date": "2018-12-21T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:44250cc7-1dcb-4719-a661-5ad3ecc39e9d", + "resource": { + "resourceType": "Immunization", + "id": "44250cc7-1dcb-4719-a661-5ad3ecc39e9d", + "status": "completed", + "date": "2018-12-21T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:76f31666-ead0-46a3-88d3-9cf48214ac39", + "resource": { + "resourceType": "DiagnosticReport", + "id": "76f31666-ead0-46a3-88d3-9cf48214ac39", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0c8e6e91-0bc4-4a48-9af0-d5f3ee417b47", + "resource": { + "resourceType": "DiagnosticReport", + "id": "0c8e6e91-0bc4-4a48-9af0-d5f3ee417b47", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" + }, + "effectiveDateTime": "2018-12-21T18:52:39-08:00", + "issued": "2018-12-21T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:abc0ddcb-9d31-4fec-b134-319d4aca0760", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:c484f502-22f3-40d3-b9f9-d095bc9834c7", + "resource": { + "resourceType": "Claim", + "id": "c484f502-22f3-40d3-b9f9-d095bc9834c7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406", + "resource": { + "resourceType": "Encounter", + "id": "de6ef63c-f784-41ef-ac59-c469e5469406", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" + } + } + ], + "period": { + "start": "2019-12-27T18:52:39-08:00", + "end": "2019-12-27T19:22:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e844227d-02ac-4c76-bf53-5798afe9282b", + "resource": { + "resourceType": "Observation", + "id": "e844227d-02ac-4c76-bf53-5798afe9282b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 163.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:edf3da83-1aea-4d79-8c68-d2b0c271b66e", + "resource": { + "resourceType": "Observation", + "id": "edf3da83-1aea-4d79-8c68-d2b0c271b66e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f8ac30f9-2105-4db5-9829-48cc6dff5746", + "resource": { + "resourceType": "Observation", + "id": "f8ac30f9-2105-4db5-9829-48cc6dff5746", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 82, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25609f0d-d738-452b-8591-5df0c13df4bc", + "resource": { + "resourceType": "Observation", + "id": "25609f0d-d738-452b-8591-5df0c13df4bc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 30.540, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51645096-b906-4001-b323-f74ea0363d2e", + "resource": { + "resourceType": "Observation", + "id": "51645096-b906-4001-b323-f74ea0363d2e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 102, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6b0b9b0-6a11-455b-a355-45e4ddccd517", + "resource": { + "resourceType": "Observation", + "id": "e6b0b9b0-6a11-455b-a355-45e4ddccd517", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e6fb00c-99fe-4cfb-b520-9f51100ce8a2", + "resource": { + "resourceType": "Observation", + "id": "4e6fb00c-99fe-4cfb-b520-9f51100ce8a2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:225841e0-8739-4d9e-a95b-0ab0f6e747a0", + "resource": { + "resourceType": "Observation", + "id": "225841e0-8739-4d9e-a95b-0ab0f6e747a0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 75.830, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d8bacb0-a294-4871-8946-6788b0baa7f6", + "resource": { + "resourceType": "Observation", + "id": "7d8bacb0-a294-4871-8946-6788b0baa7f6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 14.080, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfb6035e-6bbd-4f78-bc25-2b938e082ed7", + "resource": { + "resourceType": "Observation", + "id": "cfb6035e-6bbd-4f78-bc25-2b938e082ed7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 1.2200, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ccc75f42-5132-4251-b4bc-549497813436", + "resource": { + "resourceType": "Observation", + "id": "ccc75f42-5132-4251-b4bc-549497813436", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 9.8800, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:197848ba-270c-4424-822c-a8d9df1b6390", + "resource": { + "resourceType": "Observation", + "id": "197848ba-270c-4424-822c-a8d9df1b6390", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 136.44, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b8ee620-c378-4c26-acf0-11852056a021", + "resource": { + "resourceType": "Observation", + "id": "2b8ee620-c378-4c26-acf0-11852056a021", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 5.0800, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4646236f-3f86-4d6b-b9a2-7976dcd935f4", + "resource": { + "resourceType": "Observation", + "id": "4646236f-3f86-4d6b-b9a2-7976dcd935f4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 102.30, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bb1daea-75c1-4efe-93ca-eb2be16dfc35", + "resource": { + "resourceType": "Observation", + "id": "7bb1daea-75c1-4efe-93ca-eb2be16dfc35", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 21.710, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aa9a08a6-22eb-4685-9d7b-e91938c2b905", + "resource": { + "resourceType": "Observation", + "id": "aa9a08a6-22eb-4685-9d7b-e91938c2b905", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bec48bed-ac4f-4edd-aad8-8241711df36d", + "resource": { + "resourceType": "Observation", + "id": "bec48bed-ac4f-4edd-aad8-8241711df36d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "valueQuantity": { + "value": 5.8200, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1311b122-aed7-4ebc-bfa1-fc123ba3c6e4", + "resource": { + "resourceType": "Procedure", + "id": "1311b122-aed7-4ebc-bfa1-fc123ba3c6e4", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2019-12-27T18:52:39-08:00", + "end": "2019-12-27T19:07:39-08:00" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0cd4e24a-831a-4cb3-9b1c-2f6437e5b7ec", + "resource": { + "resourceType": "Immunization", + "id": "0cd4e24a-831a-4cb3-9b1c-2f6437e5b7ec", + "status": "completed", + "date": "2019-12-27T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d3eaf31a-f32c-4550-8acb-75312b86e276", + "resource": { + "resourceType": "Immunization", + "id": "d3eaf31a-f32c-4550-8acb-75312b86e276", + "status": "completed", + "date": "2019-12-27T18:52:39-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "33", + "display": "pneumococcal polysaccharide vaccine, 23 valent" + } + ], + "text": "pneumococcal polysaccharide vaccine, 23 valent" + }, + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b267580e-6b7e-41e2-9fd2-c3de5312fd7a", + "resource": { + "resourceType": "DiagnosticReport", + "id": "b267580e-6b7e-41e2-9fd2-c3de5312fd7a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "encounter": { + "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" + }, + "effectiveDateTime": "2019-12-27T18:52:39-08:00", + "issued": "2019-12-27T18:52:39.493-08:00", + "performer": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "result": [ + { + "reference": "urn:uuid:7bb1daea-75c1-4efe-93ca-eb2be16dfc35", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:ce3a8d7b-9307-4e1b-8eb3-1f887c0c80e4", + "resource": { + "resourceType": "Claim", + "id": "ce3a8d7b-9307-4e1b-8eb3-1f887c0c80e4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "33", + "display": "pneumococcal polysaccharide vaccine, 23 valent" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 702.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3a1dd113-8872-4a7e-b9c0-f8d2f824a81b", + "resource": { + "resourceType": "Encounter", + "id": "3a1dd113-8872-4a7e-b9c0-f8d2f824a81b", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" + } + } + ], + "period": { + "start": "2020-01-17T18:52:39-08:00", + "end": "2020-01-17T19:39:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3509b8dc-6f14-4290-a891-e3820575f275", + "resource": { + "resourceType": "Procedure", + "id": "3509b8dc-6f14-4290-a891-e3820575f275", + "subject": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "performedPeriod": { + "start": "2020-01-17T18:52:39-08:00", + "end": "2020-01-17T19:24:39-08:00" + }, + "encounter": { + "reference": "urn:uuid:3a1dd113-8872-4a7e-b9c0-f8d2f824a81b" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c5b67198-55de-4324-b547-1e6c2dbe40b1", + "resource": { + "resourceType": "Claim", + "id": "c5b67198-55de-4324-b547-1e6c2dbe40b1", + "type": "institutional", + "organization": { + "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + }, + "net": { + "value": 11482.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json new file mode 100644 index 000000000000..0acb308d86d3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json @@ -0,0 +1,10877 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7", + "resource": { + "resourceType": "Patient", + "id": "1fd2683f-2a56-47c7-b674-98f2cc8319e7", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 481861502406456147 Population seed: 1586309754086
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Race", + "code": "2106-3", + "display": "White" + } + ], + "text": "White" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Ethnicity", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + ], + "text": "Not Hispanic or Latino" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Dann525 Davis923" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Northampton", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.08312407189384233 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 43.91687592810616 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR" + } + ] + }, + "system": "http://hospital.smarthealthit.org", + "value": "1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB" + } + ] + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-31-6576" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "DL" + } + ] + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99995564" + } + ], + "name": [ + { + "use": "official", + "family": [ + "Hoppe518" + ], + "given": [ + "Elma843" + ], + "prefix": [ + "Mrs." + ] + }, + { + "use": "maiden", + "family": [ + "Luettgen772" + ], + "given": [ + "Elma843" + ], + "prefix": [ + "Mrs." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-460-2745", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1975-07-18", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.51197534529821 + }, + { + "url": "longitude", + "valueDecimal": -70.93304585681999 + } + ] + } + ], + "line": [ + "636 Brakus Station Unit 87" + ], + "city": "Salem", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "M" + } + ] + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9", + "resource": { + "resourceType": "Organization", + "id": "4861d01f-019c-3dac-a153-8334e50919f9", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "4861d01f-019c-3dac-a153-8334e50919f9" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "NORTH SHORE MEDICAL CENTER -", + "telecom": [ + { + "system": "phone", + "value": "9787411215" + } + ], + "address": [ + { + "line": [ + "81 HIGHLAND AVENUE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e", + "resource": { + "resourceType": "Practitioner", + "id": "b102f703-9e8c-37a2-8eea-08b67c680a1e", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "160" + } + ], + "active": true, + "name": { + "family": [ + "Roob72" + ], + "given": [ + "Isobel140" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "81 HIGHLAND AVENUE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:d33a3c86-5107-4f25-9130-bd7c7f7f0584", + "resource": { + "resourceType": "Encounter", + "id": "d33a3c86-5107-4f25-9130-bd7c7f7f0584", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "1990-08-30T12:47:52-07:00", + "end": "1990-08-30T13:02:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "36971009", + "display": "Sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e9c54b79-9861-4d16-8417-31c421b85020", + "resource": { + "resourceType": "Condition", + "id": "e9c54b79-9861-4d16-8417-31c421b85020", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:d33a3c86-5107-4f25-9130-bd7c7f7f0584" + }, + "dateRecorded": "1990-08-30", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "40055000", + "display": "Chronic sinusitis (disorder)" + } + ], + "text": "Chronic sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "1990-08-30T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:540c7750-af84-4144-921e-81bdea3c82ae", + "resource": { + "resourceType": "Claim", + "id": "540c7750-af84-4144-921e-81bdea3c82ae", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "40055000", + "display": "Chronic sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1eefc783-154d-4301-bdaf-53ec6c0ad698", + "resource": { + "resourceType": "Encounter", + "id": "1eefc783-154d-4301-bdaf-53ec6c0ad698", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2009-07-15T12:47:52-07:00", + "end": "2009-07-15T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ec829d7d-2c8f-4efc-ae24-968a9bfbb25c", + "resource": { + "resourceType": "MedicationOrder", + "id": "ec829d7d-2c8f-4efc-ae24-968a9bfbb25c", + "dateWritten": "2009-07-15T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:1eefc783-154d-4301-bdaf-53ec6c0ad698" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "807283", + "display": "Mirena 52 MG Intrauterine System" + } + ], + "text": "Mirena 52 MG Intrauterine System" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:5330f702-b006-40b9-86c7-e0d21dafda85", + "resource": { + "resourceType": "Claim", + "id": "5330f702-b006-40b9-86c7-e0d21dafda85", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:ec829d7d-2c8f-4efc-ae24-968a9bfbb25c" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a38ec00e-36e1-4c1a-849f-2db5e3b51f2b", + "resource": { + "resourceType": "Claim", + "id": "a38ec00e-36e1-4c1a-849f-2db5e3b51f2b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:24ed126d-6cd7-4ab5-af33-940eef487e7d", + "resource": { + "resourceType": "Encounter", + "id": "24ed126d-6cd7-4ab5-af33-940eef487e7d", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2010-04-30T12:47:52-07:00", + "end": "2010-04-30T13:13:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cf6bcedf-7d2f-47cf-a176-10570dcd40ee", + "resource": { + "resourceType": "Procedure", + "id": "cf6bcedf-7d2f-47cf-a176-10570dcd40ee", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2010-04-30T12:47:52-07:00", + "end": "2010-04-30T12:58:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:24ed126d-6cd7-4ab5-af33-940eef487e7d" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:238439bd-9910-4b3d-99ed-97e680b8693c", + "resource": { + "resourceType": "Claim", + "id": "238439bd-9910-4b3d-99ed-97e680b8693c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 4861.95, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6e6753f7-a24b-49e2-a77a-45de4bc3e3b6", + "resource": { + "resourceType": "Encounter", + "id": "6e6753f7-a24b-49e2-a77a-45de4bc3e3b6", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2010-07-30T12:47:52-07:00", + "end": "2010-07-30T13:21:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:341a008e-a687-4832-a9f8-d06b690f856a", + "resource": { + "resourceType": "Procedure", + "id": "341a008e-a687-4832-a9f8-d06b690f856a", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2010-07-30T12:47:52-07:00", + "end": "2010-07-30T13:06:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:6e6753f7-a24b-49e2-a77a-45de4bc3e3b6" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2e1406b2-2b26-4f34-8f4c-27723549b121", + "resource": { + "resourceType": "Claim", + "id": "2e1406b2-2b26-4f34-8f4c-27723549b121", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 4073.77, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:866c5085-52c9-4ecf-b791-b0e87e392ffc", + "resource": { + "resourceType": "Encounter", + "id": "866c5085-52c9-4ecf-b791-b0e87e392ffc", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2010-10-29T12:47:52-07:00", + "end": "2010-10-29T13:13:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3f3ff32b-2f53-42bd-a9b1-05ad5ee9297c", + "resource": { + "resourceType": "Procedure", + "id": "3f3ff32b-2f53-42bd-a9b1-05ad5ee9297c", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2010-10-29T12:47:52-07:00", + "end": "2010-10-29T12:58:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:866c5085-52c9-4ecf-b791-b0e87e392ffc" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4b77465b-b3ad-4b80-959d-feb3dcb92d99", + "resource": { + "resourceType": "Claim", + "id": "4b77465b-b3ad-4b80-959d-feb3dcb92d99", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 2455.95, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c34dd6dc-55e6-4246-a054-1d6deeb78195", + "resource": { + "resourceType": "Encounter", + "id": "c34dd6dc-55e6-4246-a054-1d6deeb78195", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2011-01-28T11:47:52-08:00", + "end": "2011-01-28T12:16:52-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:88ccca2b-124d-49d0-906f-c836c7f60d94", + "resource": { + "resourceType": "Procedure", + "id": "88ccca2b-124d-49d0-906f-c836c7f60d94", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2011-01-28T11:47:52-08:00", + "end": "2011-01-28T12:01:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:c34dd6dc-55e6-4246-a054-1d6deeb78195" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:eac33817-4b7b-44f1-b3ed-56d215e51fb9", + "resource": { + "resourceType": "Claim", + "id": "eac33817-4b7b-44f1-b3ed-56d215e51fb9", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 2429.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:605546ae-6d8a-4aa7-9f7d-6eba37402f65", + "resource": { + "resourceType": "Encounter", + "id": "605546ae-6d8a-4aa7-9f7d-6eba37402f65", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2011-04-29T12:47:52-07:00", + "end": "2011-04-29T13:17:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:319052d5-5d6a-408e-876b-a8a9bbcf7406", + "resource": { + "resourceType": "Procedure", + "id": "319052d5-5d6a-408e-876b-a8a9bbcf7406", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2011-04-29T12:47:52-07:00", + "end": "2011-04-29T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:605546ae-6d8a-4aa7-9f7d-6eba37402f65" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7e1060d8-1648-443d-957a-9255f6c36db0", + "resource": { + "resourceType": "Claim", + "id": "7e1060d8-1648-443d-957a-9255f6c36db0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 4145.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2f67e2fe-1b41-49e4-8a76-126cc0272375", + "resource": { + "resourceType": "Encounter", + "id": "2f67e2fe-1b41-49e4-8a76-126cc0272375", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2011-07-29T12:47:52-07:00", + "end": "2011-07-29T13:18:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:61bb7894-a14f-43ef-92eb-15cc3f8f1e6e", + "resource": { + "resourceType": "Procedure", + "id": "61bb7894-a14f-43ef-92eb-15cc3f8f1e6e", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2011-07-29T12:47:52-07:00", + "end": "2011-07-29T13:03:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:2f67e2fe-1b41-49e4-8a76-126cc0272375" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5f5d52dd-d0ba-4934-8309-79c2b9cac398", + "resource": { + "resourceType": "Claim", + "id": "5f5d52dd-d0ba-4934-8309-79c2b9cac398", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 2891.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff", + "resource": { + "resourceType": "Encounter", + "id": "b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2011-10-28T12:47:52-07:00", + "end": "2011-10-28T13:19:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4c5adff2-56ec-45e0-a95f-c8e30a02f8a1", + "resource": { + "resourceType": "Procedure", + "id": "4c5adff2-56ec-45e0-a95f-c8e30a02f8a1", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2011-10-28T12:47:52-07:00", + "end": "2011-10-28T13:04:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d402dec1-770c-4e88-9711-99db92a624cc", + "resource": { + "resourceType": "Claim", + "id": "d402dec1-770c-4e88-9711-99db92a624cc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 3754.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6b944acc-27cb-443c-885e-1176f99fc33d", + "resource": { + "resourceType": "Encounter", + "id": "6b944acc-27cb-443c-885e-1176f99fc33d", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-01-27T11:47:52-08:00", + "end": "2012-01-27T12:17:52-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6cd3de1b-f110-4446-a70c-83daae4a60e8", + "resource": { + "resourceType": "Procedure", + "id": "6cd3de1b-f110-4446-a70c-83daae4a60e8", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2012-01-27T11:47:52-08:00", + "end": "2012-01-27T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:6b944acc-27cb-443c-885e-1176f99fc33d" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:98b5c06e-96a0-4049-858b-7dacb359149d", + "resource": { + "resourceType": "Claim", + "id": "98b5c06e-96a0-4049-858b-7dacb359149d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 4855.31, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:debb7550-af72-4a4c-8fed-99adeb9888e7", + "resource": { + "resourceType": "Encounter", + "id": "debb7550-af72-4a4c-8fed-99adeb9888e7", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-04-27T12:47:52-07:00", + "end": "2012-04-27T13:14:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:53183014-7229-4db9-9727-2d8ff21a6458", + "resource": { + "resourceType": "Procedure", + "id": "53183014-7229-4db9-9727-2d8ff21a6458", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2012-04-27T12:47:52-07:00", + "end": "2012-04-27T12:59:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:debb7550-af72-4a4c-8fed-99adeb9888e7" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:70e55020-84fd-4cc9-acc5-c72b8c16057d", + "resource": { + "resourceType": "Claim", + "id": "70e55020-84fd-4cc9-acc5-c72b8c16057d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 4120.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325", + "resource": { + "resourceType": "Encounter", + "id": "93bcef01-906b-461b-8f78-0415c0508325", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-06-08T12:47:52-07:00", + "end": "2012-06-08T13:02:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e8d7f17e-8a02-42d9-9b4e-423678966a60", + "resource": { + "resourceType": "Condition", + "id": "e8d7f17e-8a02-42d9-9b4e-423678966a60", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325" + }, + "dateRecorded": "2012-06-08", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2012-06-08T12:47:52-07:00", + "abatementDateTime": "2012-06-15T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:614b3cbd-915e-41a8-8a65-204d06f0046b", + "resource": { + "resourceType": "MedicationOrder", + "id": "614b3cbd-915e-41a8-8a65-204d06f0046b", + "dateWritten": "2012-06-08T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325" + }, + "reasonReference": { + "reference": "urn:uuid:e8d7f17e-8a02-42d9-9b4e-423678966a60" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "562251", + "display": "Amoxicillin 250 MG / Clavulanate 125 MG Oral Tablet" + } + ], + "text": "Amoxicillin 250 MG / Clavulanate 125 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:00d1a2b2-b430-4748-aa19-e66ad8cd3130", + "resource": { + "resourceType": "Claim", + "id": "00d1a2b2-b430-4748-aa19-e66ad8cd3130", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:614b3cbd-915e-41a8-8a65-204d06f0046b" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ac3c1599-fa11-400b-9443-cf809263a505", + "resource": { + "resourceType": "Claim", + "id": "ac3c1599-fa11-400b-9443-cf809263a505", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cdac7ae3-4d0a-498d-bf25-a77924b07742", + "resource": { + "resourceType": "Encounter", + "id": "cdac7ae3-4d0a-498d-bf25-a77924b07742", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-07-27T12:47:52-07:00", + "end": "2012-07-27T13:17:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9654efc0-fb9f-4a8a-8ce9-eace3654387e", + "resource": { + "resourceType": "Procedure", + "id": "9654efc0-fb9f-4a8a-8ce9-eace3654387e", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "performedPeriod": { + "start": "2012-07-27T12:47:52-07:00", + "end": "2012-07-27T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cdac7ae3-4d0a-498d-bf25-a77924b07742" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:6c746559-8ab3-45e8-ac21-28bd0856532d", + "resource": { + "resourceType": "Claim", + "id": "6c746559-8ab3-45e8-ac21-28bd0856532d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + }, + "net": { + "value": 1492.94, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e", + "resource": { + "resourceType": "Organization", + "id": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "COMMONWEALTH RADIOLOGY ASSOCIATES, INC", + "telecom": [ + { + "system": "phone", + "value": "978-741-1200" + } + ], + "address": [ + { + "line": [ + "81 HIGHLAND AVE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970-2714", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc", + "resource": { + "resourceType": "Practitioner", + "id": "f4f50d8e-b084-3bc2-a787-30f3a29983fc", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "78430" + } + ], + "active": true, + "name": { + "family": [ + "Keeling57" + ], + "given": [ + "Johana303" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "81 HIGHLAND AVE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970-2714", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf", + "resource": { + "resourceType": "Encounter", + "id": "cd0dbd40-f321-4b6c-92ce-054824f3deaf", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" + } + } + ], + "period": { + "start": "2012-10-05T12:47:52-07:00", + "end": "2012-10-05T13:17:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:08f686c7-d801-4c96-8b21-3423f4731705", + "resource": { + "resourceType": "Condition", + "id": "08f686c7-d801-4c96-8b21-3423f4731705", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "dateRecorded": "2012-10-05", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2012-10-05T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:d36eb33f-bdf3-4e43-b942-365fd45e2d72", + "resource": { + "resourceType": "Observation", + "id": "d36eb33f-bdf3-4e43-b942-365fd45e2d72", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 162.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:498b2f72-3f65-4cb5-b8b7-98b9ddc8cfb2", + "resource": { + "resourceType": "Observation", + "id": "498b2f72-3f65-4cb5-b8b7-98b9ddc8cfb2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7a3a94e0-8ec3-4149-88d7-86d593abfc37", + "resource": { + "resourceType": "Observation", + "id": "7a3a94e0-8ec3-4149-88d7-86d593abfc37", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 79.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:333526be-540d-4ff3-97ac-eaf9877875cc", + "resource": { + "resourceType": "Observation", + "id": "333526be-540d-4ff3-97ac-eaf9877875cc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 30.030, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:18aa97e8-7ef3-476d-a32b-b1476b27c612", + "resource": { + "resourceType": "Observation", + "id": "18aa97e8-7ef3-476d-a32b-b1476b27c612", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e38509ed-a272-4d9b-9519-77721a420a30", + "resource": { + "resourceType": "Observation", + "id": "e38509ed-a272-4d9b-9519-77721a420a30", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:521672ec-d76c-4276-9207-543c7b24446f", + "resource": { + "resourceType": "Observation", + "id": "521672ec-d76c-4276-9207-543c7b24446f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a08f9938-f74e-4116-9a13-2abb5d785639", + "resource": { + "resourceType": "Observation", + "id": "a08f9938-f74e-4116-9a13-2abb5d785639", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 173.54, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a985217-5c0a-429a-b3d8-86207a1f86f2", + "resource": { + "resourceType": "Observation", + "id": "3a985217-5c0a-429a-b3d8-86207a1f86f2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 123.96, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6750492-22bc-40c5-a5da-6794e3139738", + "resource": { + "resourceType": "Observation", + "id": "e6750492-22bc-40c5-a5da-6794e3139738", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 70.350, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7397bc8d-08db-43dc-a30a-14f21e7cb13d", + "resource": { + "resourceType": "Observation", + "id": "7397bc8d-08db-43dc-a30a-14f21e7cb13d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueQuantity": { + "value": 78.400, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:826af47f-b98d-481e-88ee-7a375c21792a", + "resource": { + "resourceType": "Observation", + "id": "826af47f-b98d-481e-88ee-7a375c21792a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8240e5f3-9962-41f9-a9aa-db6b3254bffe", + "resource": { + "resourceType": "Procedure", + "id": "8240e5f3-9962-41f9-a9aa-db6b3254bffe", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2012-10-05T12:47:52-07:00", + "end": "2012-10-05T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:273b425d-2435-433c-b2dc-baeecc4498c4", + "resource": { + "resourceType": "Immunization", + "id": "273b425d-2435-433c-b2dc-baeecc4498c4", + "status": "completed", + "date": "2012-10-05T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:9cf84800-67b1-4016-aa14-cc4fb5aad760", + "resource": { + "resourceType": "DiagnosticReport", + "id": "9cf84800-67b1-4016-aa14-cc4fb5aad760", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" + }, + "effectiveDateTime": "2012-10-05T12:47:52-07:00", + "issued": "2012-10-05T12:47:52.730-07:00", + "performer": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "result": [ + { + "reference": "urn:uuid:7397bc8d-08db-43dc-a30a-14f21e7cb13d", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:3d2ad77e-b88d-457a-9574-de0d4ec5e25c", + "resource": { + "resourceType": "Claim", + "id": "3d2ad77e-b88d-457a-9574-de0d4ec5e25c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 1007.40, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8", + "resource": { + "resourceType": "Encounter", + "id": "d0e4fada-dc01-4853-8bd2-f8c45e6455a8", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T18:02:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f", + "resource": { + "resourceType": "Condition", + "id": "72e7d27a-a498-407c-afb8-6fd1b50b1a8f", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + }, + "dateRecorded": "2012-10-19", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ], + "text": "Normal pregnancy" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2012-10-19T12:47:52-07:00", + "abatementDateTime": "2013-05-17T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ee3fd7c0-f1c7-4f07-9e93-46eb5a4e99ff", + "resource": { + "resourceType": "Procedure", + "id": "ee3fd7c0-f1c7-4f07-9e93-46eb5a4e99ff", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "252160004", + "display": "Standard pregnancy test" + } + ], + "text": "Standard pregnancy test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:05cf5d86-32cb-4e39-9b98-ceeeba210eed", + "resource": { + "resourceType": "Procedure", + "id": "05cf5d86-32cb-4e39-9b98-ceeeba210eed", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "169230002", + "display": "Ultrasound scan for fetal viability" + } + ], + "text": "Ultrasound scan for fetal viability" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:98e52fb8-ad50-4459-b782-d479e050cab5", + "resource": { + "resourceType": "Procedure", + "id": "98e52fb8-ad50-4459-b782-d479e050cab5", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:38ef00c7-d111-4537-9c0b-772f25b91941", + "resource": { + "resourceType": "Procedure", + "id": "38ef00c7-d111-4537-9c0b-772f25b91941", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:8089eaea-d373-4c02-af87-794a39058207", + "resource": { + "resourceType": "Procedure", + "id": "8089eaea-d373-4c02-af87-794a39058207", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44608003", + "display": "Blood typing, RH typing" + } + ], + "text": "Blood typing, RH typing" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c96c9f12-6f03-462a-b03c-ef317e417d79", + "resource": { + "resourceType": "Procedure", + "id": "c96c9f12-6f03-462a-b03c-ef317e417d79", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104091002", + "display": "Hemoglobin / Hematocrit / Platelet count" + } + ], + "text": "Hemoglobin / Hematocrit / Platelet count" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:425f4eef-68f8-4d75-a899-21b09fe0ad19", + "resource": { + "resourceType": "Procedure", + "id": "425f4eef-68f8-4d75-a899-21b09fe0ad19", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "47758006", + "display": "Hepatitis B Surface Antigen Measurement" + } + ], + "text": "Hepatitis B Surface Antigen Measurement" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f68721ca-4410-4e4f-87d5-b2424ef5b129", + "resource": { + "resourceType": "Procedure", + "id": "f68721ca-4410-4e4f-87d5-b2424ef5b129", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "31676001", + "display": "Human immunodeficiency virus antigen test" + } + ], + "text": "Human immunodeficiency virus antigen test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2446e543-84b3-48e5-829f-a3ff9f5e3d98", + "resource": { + "resourceType": "Procedure", + "id": "2446e543-84b3-48e5-829f-a3ff9f5e3d98", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "310861008", + "display": "Chlamydia antigen test" + } + ], + "text": "Chlamydia antigen test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b6628389-c214-4890-88e6-7f15a3701212", + "resource": { + "resourceType": "Procedure", + "id": "b6628389-c214-4890-88e6-7f15a3701212", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "165829005", + "display": "Gonorrhea infection test" + } + ], + "text": "Gonorrhea infection test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:eaf71f03-b202-4f92-a7ce-70c9154f64cc", + "resource": { + "resourceType": "Procedure", + "id": "eaf71f03-b202-4f92-a7ce-70c9154f64cc", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269828009", + "display": "Syphilis infection test" + } + ], + "text": "Syphilis infection test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b7f227b6-4d16-4148-96e3-dac95b399c84", + "resource": { + "resourceType": "Procedure", + "id": "b7f227b6-4d16-4148-96e3-dac95b399c84", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117010004", + "display": "Urine culture" + } + ], + "text": "Urine culture" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:30ec60ab-4285-4784-bfb8-c60c552edd91", + "resource": { + "resourceType": "Procedure", + "id": "30ec60ab-4285-4784-bfb8-c60c552edd91", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "90226004", + "display": "Cytopathology procedure, preparation of smear, genital source" + } + ], + "text": "Cytopathology procedure, preparation of smear, genital source" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0e0acfbd-8e13-4068-be5b-c8db28aa8370", + "resource": { + "resourceType": "Procedure", + "id": "0e0acfbd-8e13-4068-be5b-c8db28aa8370", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "395123002", + "display": "Urine screening test for diabetes" + } + ], + "text": "Urine screening test for diabetes" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4befc1a0-2733-4e67-be8e-6e28f3607208", + "resource": { + "resourceType": "Procedure", + "id": "4befc1a0-2733-4e67-be8e-6e28f3607208", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104375008", + "display": "Hepatitis C antibody test" + } + ], + "text": "Hepatitis C antibody test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4fdf0764-5136-4bfc-b05b-4fc7eb43277f", + "resource": { + "resourceType": "Procedure", + "id": "4fdf0764-5136-4bfc-b05b-4fc7eb43277f", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "169690007", + "display": "Rubella screening" + } + ], + "text": "Rubella screening" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:97ea62c5-f862-4e24-8f49-6d856bade41b", + "resource": { + "resourceType": "Procedure", + "id": "97ea62c5-f862-4e24-8f49-6d856bade41b", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104326007", + "display": "Measurement of Varicella-zoster virus antibody" + } + ], + "text": "Measurement of Varicella-zoster virus antibody" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0d6f1573-9c15-46bc-a3be-75016463add0", + "resource": { + "resourceType": "Procedure", + "id": "0d6f1573-9c15-46bc-a3be-75016463add0", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "28163009", + "display": "Skin test for tuberculosis" + } + ], + "text": "Skin test for tuberculosis" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b3cced7a-ef97-4147-b87d-1b9d79d30142", + "resource": { + "resourceType": "Procedure", + "id": "b3cced7a-ef97-4147-b87d-1b9d79d30142", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "167271000", + "display": "Urine protein test" + } + ], + "text": "Urine protein test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:532d6177-9796-4068-9cda-f05c143bf578", + "resource": { + "resourceType": "Procedure", + "id": "532d6177-9796-4068-9cda-f05c143bf578", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination of mother" + } + ], + "text": "Physical examination of mother" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2012-10-19T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ac3a8152-5640-473d-8609-a6896c30f6c9", + "resource": { + "resourceType": "CarePlan", + "id": "ac3a8152-5640-473d-8609-a6896c30f6c9", + "text": { + "status": "generated", + "div": "
    Routine antenatal care
    " + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "context": { + "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" + }, + "period": { + "start": "2012-10-19T12:47:52-07:00", + "end": "2013-05-17T12:47:52-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "134435003", + "display": "Routine antenatal care" + } + ], + "text": "Routine antenatal care" + } + ], + "addresses": [ + { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "135892000", + "display": "Antenatal education" + } + ], + "text": "Antenatal education" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "713076009", + "display": "Antenatal risk assessment" + } + ], + "text": "Antenatal risk assessment" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312404004", + "display": "Antenatal blood tests" + } + ], + "text": "Antenatal blood tests" + }, + "status": "completed", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:5708f080-1539-4c2d-b6a4-fae148c86357", + "resource": { + "resourceType": "Claim", + "id": "5708f080-1539-4c2d-b6a4-fae148c86357", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "252160004", + "display": "Standard pregnancy test" + }, + "net": { + "value": 5212.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "169230002", + "display": "Ultrasound scan for fetal viability" + }, + "net": { + "value": 15228.25, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 9562.18, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 7670.03, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "44608003", + "display": "Blood typing, RH typing" + }, + "net": { + "value": 1389.71, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 8, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "104091002", + "display": "Hemoglobin / Hematocrit / Platelet count" + }, + "net": { + "value": 2108.07, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 9, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "47758006", + "display": "Hepatitis B Surface Antigen Measurement" + }, + "net": { + "value": 2914.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 10, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "31676001", + "display": "Human immunodeficiency virus antigen test" + }, + "net": { + "value": 2848.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 11, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "310861008", + "display": "Chlamydia antigen test" + }, + "net": { + "value": 2512.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 12, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "165829005", + "display": "Gonorrhea infection test" + }, + "net": { + "value": 2714.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 13, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "269828009", + "display": "Syphilis infection test" + }, + "net": { + "value": 2136.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 14, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "117010004", + "display": "Urine culture" + }, + "net": { + "value": 2853.03, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 15, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "90226004", + "display": "Cytopathology procedure, preparation of smear, genital source" + }, + "net": { + "value": 2717.85, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 16, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "395123002", + "display": "Urine screening test for diabetes" + }, + "net": { + "value": 2950.24, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 17, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "104375008", + "display": "Hepatitis C antibody test" + }, + "net": { + "value": 2176.72, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 18, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "169690007", + "display": "Rubella screening" + }, + "net": { + "value": 2229.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 19, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "104326007", + "display": "Measurement of Varicella-zoster virus antibody" + }, + "net": { + "value": 2864.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 20, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "28163009", + "display": "Skin test for tuberculosis" + }, + "net": { + "value": 1485.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 21, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "167271000", + "display": "Urine protein test" + }, + "net": { + "value": 2916.90, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 22, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination of mother" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4", + "resource": { + "resourceType": "Encounter", + "id": "4153dbe1-f877-43eb-8498-627b47dbabe4", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-11-16T11:47:52-08:00", + "end": "2012-11-16T12:47:52-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a2de3620-89d7-4f5e-83a9-43ef8ae2410a", + "resource": { + "resourceType": "Procedure", + "id": "a2de3620-89d7-4f5e-83a9-43ef8ae2410a", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-11-16T11:47:52-08:00", + "end": "2012-11-16T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4293be80-fa68-455f-9d46-c71ea4279544", + "resource": { + "resourceType": "Procedure", + "id": "4293be80-fa68-455f-9d46-c71ea4279544", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-11-16T11:47:52-08:00", + "end": "2012-11-16T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3b1d7021-63c6-4e72-bc8f-af7a6313989e", + "resource": { + "resourceType": "Procedure", + "id": "3b1d7021-63c6-4e72-bc8f-af7a6313989e", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "443529005", + "display": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" + } + ], + "text": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-11-16T11:47:52-08:00", + "end": "2012-11-16T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b908fca2-c9b4-4dc7-beb8-d915d358e550", + "resource": { + "resourceType": "Claim", + "id": "b908fca2-c9b4-4dc7-beb8-d915d358e550", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 7348.06, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 9287.00, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "443529005", + "display": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" + }, + "net": { + "value": 1654.54, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2", + "resource": { + "resourceType": "Encounter", + "id": "42034ba4-ed69-459c-b832-6c15bdefc7c2", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2012-12-14T11:47:52-08:00", + "end": "2012-12-14T13:02:52-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a90044b0-8810-4b11-b628-2501c650dd4d", + "resource": { + "resourceType": "Procedure", + "id": "a90044b0-8810-4b11-b628-2501c650dd4d", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271442007", + "display": "Fetal anatomy study" + } + ], + "text": "Fetal anatomy study" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-12-14T11:47:52-08:00", + "end": "2012-12-14T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a8534dc7-6e36-44f1-b7bd-32db0946e08f", + "resource": { + "resourceType": "Procedure", + "id": "a8534dc7-6e36-44f1-b7bd-32db0946e08f", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "275833003", + "display": "Alpha-fetoprotein test" + } + ], + "text": "Alpha-fetoprotein test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-12-14T11:47:52-08:00", + "end": "2012-12-14T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:17ea9dcb-cff1-49ff-af65-d0a2ba16ffcc", + "resource": { + "resourceType": "Procedure", + "id": "17ea9dcb-cff1-49ff-af65-d0a2ba16ffcc", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-12-14T11:47:52-08:00", + "end": "2012-12-14T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c0576791-0d92-4b94-a133-5f6e789c614d", + "resource": { + "resourceType": "Procedure", + "id": "c0576791-0d92-4b94-a133-5f6e789c614d", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2012-12-14T11:47:52-08:00", + "end": "2012-12-14T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:344aff95-7921-4fa6-8595-79addeccc669", + "resource": { + "resourceType": "Claim", + "id": "344aff95-7921-4fa6-8595-79addeccc669", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "271442007", + "display": "Fetal anatomy study" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "275833003", + "display": "Alpha-fetoprotein test" + }, + "net": { + "value": 1929.30, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 6891.86, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 6656.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d", + "resource": { + "resourceType": "Encounter", + "id": "ed2fcb69-b3f9-4298-bfe2-e01c331cef8d", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-01-11T11:47:52-08:00", + "end": "2013-01-11T12:32:52-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:eb158c5e-0c6d-4748-a302-35eafa26752b", + "resource": { + "resourceType": "Procedure", + "id": "eb158c5e-0c6d-4748-a302-35eafa26752b", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-01-11T11:47:52-08:00", + "end": "2013-01-11T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:52b94e1a-141f-4930-be60-4c6361eb6108", + "resource": { + "resourceType": "Procedure", + "id": "52b94e1a-141f-4930-be60-4c6361eb6108", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-01-11T11:47:52-08:00", + "end": "2013-01-11T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:44e2a39e-0d74-4bdc-b577-3217f7db7550", + "resource": { + "resourceType": "Claim", + "id": "44e2a39e-0d74-4bdc-b577-3217f7db7550", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 7288.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 5248.44, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093", + "resource": { + "resourceType": "Encounter", + "id": "a40ac6e5-fef8-4875-a8d5-709953a76093", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-02-08T11:47:52-08:00", + "end": "2013-02-08T12:32:52-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d606b989-27ec-40eb-b6a2-eb23cb5281a9", + "resource": { + "resourceType": "Procedure", + "id": "d606b989-27ec-40eb-b6a2-eb23cb5281a9", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-02-08T11:47:52-08:00", + "end": "2013-02-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f6274adb-25b2-4de4-b36d-c30cc1e5a40c", + "resource": { + "resourceType": "Procedure", + "id": "f6274adb-25b2-4de4-b36d-c30cc1e5a40c", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-02-08T11:47:52-08:00", + "end": "2013-02-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b3ff90e4-9960-4694-aa63-6dab8fbac5f0", + "resource": { + "resourceType": "Claim", + "id": "b3ff90e4-9960-4694-aa63-6dab8fbac5f0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 6444.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 5708.97, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907", + "resource": { + "resourceType": "Encounter", + "id": "88d42111-285b-4199-8000-a10a199a9907", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T13:17:52-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3d506b4c-cd43-4d04-a874-0835cfc28358", + "resource": { + "resourceType": "Procedure", + "id": "3d506b4c-cd43-4d04-a874-0835cfc28358", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104091002", + "display": "Hemoglobin / Hematocrit / Platelet count" + } + ], + "text": "Hemoglobin / Hematocrit / Platelet count" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:eacdbd19-ba24-4dbf-93d4-016282aee0d1", + "resource": { + "resourceType": "Procedure", + "id": "eacdbd19-ba24-4dbf-93d4-016282aee0d1", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "399014008", + "display": "Vaccination for diphtheria, pertussis, and tetanus" + } + ], + "text": "Vaccination for diphtheria, pertussis, and tetanus" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:33b20d57-35ee-498f-b49a-cd3867b88747", + "resource": { + "resourceType": "Procedure", + "id": "33b20d57-35ee-498f-b49a-cd3867b88747", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "268556000", + "display": "Urine screening for glucose" + } + ], + "text": "Urine screening for glucose" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9f5d2bae-f114-4ebd-8316-bf9b713b8217", + "resource": { + "resourceType": "Procedure", + "id": "9f5d2bae-f114-4ebd-8316-bf9b713b8217", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3e565b4d-3353-43aa-a7b1-6e4e98af87b4", + "resource": { + "resourceType": "Procedure", + "id": "3e565b4d-3353-43aa-a7b1-6e4e98af87b4", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-03-08T11:47:52-08:00", + "end": "2013-03-08T12:02:52-08:00" + }, + "encounter": { + "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d53524f0-b28e-45e4-a40a-20558b9cc51d", + "resource": { + "resourceType": "Claim", + "id": "d53524f0-b28e-45e4-a40a-20558b9cc51d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "104091002", + "display": "Hemoglobin / Hematocrit / Platelet count" + }, + "net": { + "value": 1837.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "399014008", + "display": "Vaccination for diphtheria, pertussis, and tetanus" + }, + "net": { + "value": 3667.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "268556000", + "display": "Urine screening for glucose" + }, + "net": { + "value": 3248.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 10305.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 7134.31, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428", + "resource": { + "resourceType": "Encounter", + "id": "5c17fe8f-881b-4071-a2b6-28a10ab25428", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-04-05T12:47:52-07:00", + "end": "2013-04-05T13:32:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:da4576b1-9b56-4168-8f83-7388dd6d3b06", + "resource": { + "resourceType": "Procedure", + "id": "da4576b1-9b56-4168-8f83-7388dd6d3b06", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-04-05T12:47:52-07:00", + "end": "2013-04-05T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e9d98b4d-e178-47c8-ba7f-ddfd3bcc1f4e", + "resource": { + "resourceType": "Procedure", + "id": "e9d98b4d-e178-47c8-ba7f-ddfd3bcc1f4e", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-04-05T12:47:52-07:00", + "end": "2013-04-05T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:59505f7c-1995-41d7-88f1-7fec7f002971", + "resource": { + "resourceType": "Claim", + "id": "59505f7c-1995-41d7-88f1-7fec7f002971", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 7325.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 10230.59, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c", + "resource": { + "resourceType": "Encounter", + "id": "fd5881ab-1d21-4aca-b59c-08305ee9079c", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-05-03T12:47:52-07:00", + "end": "2013-05-03T13:47:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d73b252f-2b8b-4e50-9ff4-e8177f090151", + "resource": { + "resourceType": "Procedure", + "id": "d73b252f-2b8b-4e50-9ff4-e8177f090151", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "118001005", + "display": "Streptococcus pneumoniae group B antigen test" + } + ], + "text": "Streptococcus pneumoniae group B antigen test" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-05-03T12:47:52-07:00", + "end": "2013-05-03T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d4c98a0f-3057-4763-adca-33d1728e08cd", + "resource": { + "resourceType": "Procedure", + "id": "d4c98a0f-3057-4763-adca-33d1728e08cd", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + } + ], + "text": "Evaluation of uterine fundal height" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-05-03T12:47:52-07:00", + "end": "2013-05-03T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:dddbbe44-e447-46cc-8fa5-8c0872de32ee", + "resource": { + "resourceType": "Procedure", + "id": "dddbbe44-e447-46cc-8fa5-8c0872de32ee", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + } + ], + "text": "Auscultation of the fetal heart" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-05-03T12:47:52-07:00", + "end": "2013-05-03T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:792a9ad3-a723-469a-a219-13495fb3b3a2", + "resource": { + "resourceType": "Claim", + "id": "792a9ad3-a723-469a-a219-13495fb3b3a2", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "118001005", + "display": "Streptococcus pneumoniae group B antigen test" + }, + "net": { + "value": 1207.97, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274804006", + "display": "Evaluation of uterine fundal height" + }, + "net": { + "value": 8244.47, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "225158009", + "display": "Auscultation of the fetal heart" + }, + "net": { + "value": 4228.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80", + "resource": { + "resourceType": "Encounter", + "id": "d51c12d4-eeab-4c80-b5bc-b1dc14235c80", + "status": "finished", + "class": "emergency", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "183460006", + "display": "Obstetric emergency hospital admission" + } + ], + "text": "Obstetric emergency hospital admission" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-05-17T12:47:52-07:00", + "end": "2013-05-17T14:17:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e44ef6b9-aa50-4296-93f7-021839e65728", + "resource": { + "resourceType": "Procedure", + "id": "e44ef6b9-aa50-4296-93f7-021839e65728", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "31208007", + "display": "Medical induction of labor" + } + ], + "text": "Medical induction of labor" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-05-17T12:47:52-07:00", + "end": "2013-05-17T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7f9d558f-e196-4c0c-b951-92ed7da439b3", + "resource": { + "resourceType": "Procedure", + "id": "7f9d558f-e196-4c0c-b951-92ed7da439b3", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "11466000", + "display": "Cesarean section" + } + ], + "text": "Cesarean section" + }, + "reasonReference": { + "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" + }, + "performedPeriod": { + "start": "2013-05-17T12:47:52-07:00", + "end": "2013-05-17T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:924eb1bd-bf53-4118-a423-36b964bae627", + "resource": { + "resourceType": "Claim", + "id": "924eb1bd-bf53-4118-a423-36b964bae627", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "183460006", + "display": "Obstetric emergency hospital admission" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "31208007", + "display": "Medical induction of labor" + }, + "net": { + "value": 4340.81, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "11466000", + "display": "Cesarean section" + }, + "net": { + "value": 9307.60, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154", + "resource": { + "resourceType": "Encounter", + "id": "014a64da-89fc-4a0b-b2c1-7f6b589da154", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "169762003", + "display": "Postnatal visit" + } + ], + "text": "Postnatal visit" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-06-28T12:47:52-07:00", + "end": "2013-06-28T13:32:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:644d62c7-30a9-4eb7-84b5-0763637ebb17", + "resource": { + "resourceType": "Procedure", + "id": "644d62c7-30a9-4eb7-84b5-0763637ebb17", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination following birth" + } + ], + "text": "Physical examination following birth" + }, + "performedPeriod": { + "start": "2013-06-28T12:47:52-07:00", + "end": "2013-06-28T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:15d7b689-dedc-4132-b6ae-465f68b7bf82", + "resource": { + "resourceType": "Procedure", + "id": "15d7b689-dedc-4132-b6ae-465f68b7bf82", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "171207006", + "display": "Depression screening" + } + ], + "text": "Depression screening" + }, + "performedPeriod": { + "start": "2013-06-28T12:47:52-07:00", + "end": "2013-06-28T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c797851e-eaae-4b8c-8ac6-95b2cf6563da", + "resource": { + "resourceType": "Claim", + "id": "c797851e-eaae-4b8c-8ac6-95b2cf6563da", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "169762003", + "display": "Postnatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination following birth" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "171207006", + "display": "Depression screening" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427", + "resource": { + "resourceType": "Encounter", + "id": "1a862e3a-bd75-4182-bb4f-924462d76427", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-07-03T12:47:52-07:00", + "end": "2013-07-03T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a7ccf351-7ba6-4dee-a64c-8a81a715054a", + "resource": { + "resourceType": "Condition", + "id": "a7ccf351-7ba6-4dee-a64c-8a81a715054a", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427" + }, + "dateRecorded": "2013-07-08", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2013-07-08T12:47:52-07:00", + "abatementDateTime": "2013-07-08T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:e7f54cbf-af1a-4798-bc8c-47a1abce8645", + "resource": { + "resourceType": "MedicationOrder", + "id": "e7f54cbf-af1a-4798-bc8c-47a1abce8645", + "dateWritten": "2013-07-03T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "748962", + "display": "Camila 28 Day Pack" + } + ], + "text": "Camila 28 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:aaf2302b-2687-453e-bf5e-cf002712b3f8", + "resource": { + "resourceType": "Claim", + "id": "aaf2302b-2687-453e-bf5e-cf002712b3f8", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:e7f54cbf-af1a-4798-bc8c-47a1abce8645" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:69b20101-431d-45e6-b688-f7b65c7be1c3", + "resource": { + "resourceType": "Claim", + "id": "69b20101-431d-45e6-b688-f7b65c7be1c3", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9", + "resource": { + "resourceType": "Encounter", + "id": "cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2013-07-08T12:47:52-07:00", + "end": "2013-07-08T14:03:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:dfbe2e5f-246e-49c5-a9a8-dd5941574228", + "resource": { + "resourceType": "Observation", + "id": "dfbe2e5f-246e-49c5-a9a8-dd5941574228", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 11.026, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:26f50d1b-3f8a-4ca2-a44c-13528fd7761d", + "resource": { + "resourceType": "Observation", + "id": "26f50d1b-3f8a-4ca2-a44c-13528fd7761d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20570-8", + "display": "Hematocrit [Volume Fraction] of Blood" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 35.399, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb2b41d8-2b99-41e0-b9a9-17b39f7056b0", + "resource": { + "resourceType": "Observation", + "id": "fb2b41d8-2b99-41e0-b9a9-17b39f7056b0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "WBC Auto (Bld) [#/Vol]" + } + ], + "text": "WBC Auto (Bld) [#/Vol]" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 7.8841, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8afdd8e-e6c6-414f-b6bc-4c732b4e55c8", + "resource": { + "resourceType": "Observation", + "id": "d8afdd8e-e6c6-414f-b6bc-4c732b4e55c8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "RBC Auto (Bld) [#/Vol]" + } + ], + "text": "RBC Auto (Bld) [#/Vol]" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 5.0896, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c867668-9947-4bbd-8745-03cb2ca64f30", + "resource": { + "resourceType": "Observation", + "id": "3c867668-9947-4bbd-8745-03cb2ca64f30", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 88.709, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b67eca8a-856e-4d9f-8c93-b4d1c4d913c2", + "resource": { + "resourceType": "Observation", + "id": "b67eca8a-856e-4d9f-8c93-b4d1c4d913c2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 29.414, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7fe15fab-c5ef-4ca5-a4a7-1a7735be85e1", + "resource": { + "resourceType": "Observation", + "id": "7fe15fab-c5ef-4ca5-a4a7-1a7735be85e1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 33.768, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5540aed0-861a-41c3-8e96-09c220ee0b27", + "resource": { + "resourceType": "Observation", + "id": "5540aed0-861a-41c3-8e96-09c220ee0b27", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "RDW - Erythrocyte distribution width Auto (RBC) [Entitic vol]" + } + ], + "text": "RDW - Erythrocyte distribution width Auto (RBC) [Entitic vol]" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 42.075, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:784af967-65b2-4d1b-b6f0-222df46bf033", + "resource": { + "resourceType": "Observation", + "id": "784af967-65b2-4d1b-b6f0-222df46bf033", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 305.64, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:19c1e4da-f18c-4421-8a7f-6aac97530f10", + "resource": { + "resourceType": "Observation", + "id": "19c1e4da-f18c-4421-8a7f-6aac97530f10", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 517.81, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:578d9ea5-43fb-4f5a-aea4-cdd190e094b6", + "resource": { + "resourceType": "Observation", + "id": "578d9ea5-43fb-4f5a-aea4-cdd190e094b6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "valueQuantity": { + "value": 11.873, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2a901aa2-8934-4452-99e2-2554933793e2", + "resource": { + "resourceType": "Procedure", + "id": "2a901aa2-8934-4452-99e2-2554933793e2", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "14768001", + "display": "Peripheral blood smear interpretation" + } + ], + "text": "Peripheral blood smear interpretation" + }, + "performedPeriod": { + "start": "2013-07-08T12:47:52-07:00", + "end": "2013-07-08T13:17:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:844b8b17-c7a4-4519-93b8-da01266836bf", + "resource": { + "resourceType": "Procedure", + "id": "844b8b17-c7a4-4519-93b8-da01266836bf", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "415300000", + "display": "Review of systems (procedure)" + } + ], + "text": "Review of systems (procedure)" + }, + "performedPeriod": { + "start": "2013-07-08T12:47:52-07:00", + "end": "2013-07-08T12:57:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7b812be3-2ec0-4f6d-ad4a-488011b5d315", + "resource": { + "resourceType": "Procedure", + "id": "7b812be3-2ec0-4f6d-ad4a-488011b5d315", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2013-07-08T12:47:52-07:00", + "end": "2013-07-08T12:55:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a4d661fe-489b-4acd-9a59-f530a23df565", + "resource": { + "resourceType": "Procedure", + "id": "a4d661fe-489b-4acd-9a59-f530a23df565", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162676008", + "display": "Brief general examination (procedure)" + } + ], + "text": "Brief general examination (procedure)" + }, + "performedPeriod": { + "start": "2013-07-08T12:47:52-07:00", + "end": "2013-07-08T13:00:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a9cb7d8a-de37-4f1d-88f4-8723c218f7e0", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a9cb7d8a-de37-4f1d-88f4-8723c218f7e0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" + }, + "effectiveDateTime": "2013-07-08T12:47:52-07:00", + "issued": "2013-07-08T12:47:52.730-07:00", + "performer": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "result": [ + { + "reference": "urn:uuid:578d9ea5-43fb-4f5a-aea4-cdd190e094b6", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0c435b76-697a-4514-8d0a-0a5a42b4e0e7", + "resource": { + "resourceType": "Claim", + "id": "0c435b76-697a-4514-8d0a-0a5a42b4e0e7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "14768001", + "display": "Peripheral blood smear interpretation" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "415300000", + "display": "Review of systems (procedure)" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 469.21, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162676008", + "display": "Brief general examination (procedure)" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6b7383a6-0ded-408e-8701-d1e72f17697a", + "resource": { + "resourceType": "Encounter", + "id": "6b7383a6-0ded-408e-8701-d1e72f17697a", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2014-06-28T12:47:52-07:00", + "end": "2014-06-28T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d", + "resource": { + "resourceType": "MedicationOrder", + "id": "0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d", + "dateWritten": "2014-06-28T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:6b7383a6-0ded-408e-8701-d1e72f17697a" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "757594", + "display": "Jolivette 28 Day Pack" + } + ], + "text": "Jolivette 28 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:0cb4cac2-eb7d-4da1-b120-22df6a12f592", + "resource": { + "resourceType": "Claim", + "id": "0cb4cac2-eb7d-4da1-b120-22df6a12f592", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8a26487b-9cfd-425b-b3aa-260e7c21862a", + "resource": { + "resourceType": "Claim", + "id": "8a26487b-9cfd-425b-b3aa-260e7c21862a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e19a6916-47ef-433a-b37a-1c4ff1f2729a", + "resource": { + "resourceType": "Encounter", + "id": "e19a6916-47ef-433a-b37a-1c4ff1f2729a", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2015-06-23T12:47:52-07:00", + "end": "2015-06-23T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c9f6b596-614a-456b-8133-de75146f8710", + "resource": { + "resourceType": "MedicationOrder", + "id": "c9f6b596-614a-456b-8133-de75146f8710", + "dateWritten": "2015-06-23T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:e19a6916-47ef-433a-b37a-1c4ff1f2729a" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1367439", + "display": "NuvaRing 0.12/0.015 MG per 24HR 21 Day Vaginal Ring" + } + ], + "text": "NuvaRing 0.12/0.015 MG per 24HR 21 Day Vaginal Ring" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:d24a3c2a-24ad-48da-b720-c04d2c58ece4", + "resource": { + "resourceType": "Claim", + "id": "d24a3c2a-24ad-48da-b720-c04d2c58ece4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:c9f6b596-614a-456b-8133-de75146f8710" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:214e96bb-b248-4a18-a0d0-4c71bf3fc1e5", + "resource": { + "resourceType": "Claim", + "id": "214e96bb-b248-4a18-a0d0-4c71bf3fc1e5", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642", + "resource": { + "resourceType": "Encounter", + "id": "e396cffd-247d-4fdd-858a-5fa85574c642", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" + } + } + ], + "period": { + "start": "2015-07-24T12:47:52-07:00", + "end": "2015-07-24T13:17:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f574d297-e04a-482a-ab71-198be984cdb2", + "resource": { + "resourceType": "Observation", + "id": "f574d297-e04a-482a-ab71-198be984cdb2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 162.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:370f86e4-09cb-4ede-8a32-a40b0c6930b3", + "resource": { + "resourceType": "Observation", + "id": "370f86e4-09cb-4ede-8a32-a40b0c6930b3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30f7d65b-95ae-42a2-ab36-972bef42df71", + "resource": { + "resourceType": "Observation", + "id": "30f7d65b-95ae-42a2-ab36-972bef42df71", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 79.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5066aa49-ccfb-425a-8c1d-6a5a04eb0b16", + "resource": { + "resourceType": "Observation", + "id": "5066aa49-ccfb-425a-8c1d-6a5a04eb0b16", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 30.030, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb62634b-be80-42ef-951d-01306a1bd9e3", + "resource": { + "resourceType": "Observation", + "id": "cb62634b-be80-42ef-951d-01306a1bd9e3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4cd15784-77b5-4dc3-8f3a-8562c27329d5", + "resource": { + "resourceType": "Observation", + "id": "4cd15784-77b5-4dc3-8f3a-8562c27329d5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 88, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31abcec9-89cd-441d-9c6e-755149c94baa", + "resource": { + "resourceType": "Observation", + "id": "31abcec9-89cd-441d-9c6e-755149c94baa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d5ed4719-611c-4ff2-b79b-0675dc590183", + "resource": { + "resourceType": "Observation", + "id": "d5ed4719-611c-4ff2-b79b-0675dc590183", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 3.7063, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbec67f1-5e49-4a27-bc2f-7c38f9bee7df", + "resource": { + "resourceType": "Observation", + "id": "bbec67f1-5e49-4a27-bc2f-7c38f9bee7df", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 4.5130, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d10dabf-8491-4775-a4e8-9d05945a9d76", + "resource": { + "resourceType": "Observation", + "id": "3d10dabf-8491-4775-a4e8-9d05945a9d76", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 14.989, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b7152cc-b9d9-4eaf-98d0-81682aac404b", + "resource": { + "resourceType": "Observation", + "id": "2b7152cc-b9d9-4eaf-98d0-81682aac404b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 41.010, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65c103ab-e6d1-442e-a710-721df8495e1d", + "resource": { + "resourceType": "Observation", + "id": "65c103ab-e6d1-442e-a710-721df8495e1d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 89.560, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82f24393-cea1-4342-9fb3-c4845136a5e7", + "resource": { + "resourceType": "Observation", + "id": "82f24393-cea1-4342-9fb3-c4845136a5e7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 27.456, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0368e7cf-3b1f-413d-950a-89a2c370cc2d", + "resource": { + "resourceType": "Observation", + "id": "0368e7cf-3b1f-413d-950a-89a2c370cc2d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 35.103, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33e621d9-4bc2-4f69-9987-c93786e01970", + "resource": { + "resourceType": "Observation", + "id": "33e621d9-4bc2-4f69-9987-c93786e01970", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 40.945, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7a07912-88a6-4ce6-89b2-d467ac17fd2f", + "resource": { + "resourceType": "Observation", + "id": "b7a07912-88a6-4ce6-89b2-d467ac17fd2f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 167.85, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2cff4c7-a35b-4be1-9cd4-e06840213f3d", + "resource": { + "resourceType": "Observation", + "id": "b2cff4c7-a35b-4be1-9cd4-e06840213f3d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 487.14, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d000ca8-1d46-4c7e-a5a1-4dede883d961", + "resource": { + "resourceType": "Observation", + "id": "1d000ca8-1d46-4c7e-a5a1-4dede883d961", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueQuantity": { + "value": 11.167, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:79653c9e-a2d2-42c1-b765-7da6f45ed748", + "resource": { + "resourceType": "Observation", + "id": "79653c9e-a2d2-42c1-b765-7da6f45ed748", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8d07f36d-450c-4f9a-9809-a98f792bfa5f", + "resource": { + "resourceType": "Procedure", + "id": "8d07f36d-450c-4f9a-9809-a98f792bfa5f", + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2015-07-24T12:47:52-07:00", + "end": "2015-07-24T13:02:52-07:00" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ba21d7c7-2d01-43ed-bf69-c7ff9b6a5a97", + "resource": { + "resourceType": "Immunization", + "id": "ba21d7c7-2d01-43ed-bf69-c7ff9b6a5a97", + "status": "completed", + "date": "2015-07-24T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:bb99b956-fc6c-4023-9745-3c84b96c41cd", + "resource": { + "resourceType": "Immunization", + "id": "bb99b956-fc6c-4023-9745-3c84b96c41cd", + "status": "completed", + "date": "2015-07-24T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + } + ], + "text": "Hep A, adult" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a70d175a-b888-41e1-b0c6-03c6c16097b9", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a70d175a-b888-41e1-b0c6-03c6c16097b9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" + }, + "effectiveDateTime": "2015-07-24T12:47:52-07:00", + "issued": "2015-07-24T12:47:52.730-07:00", + "performer": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "result": [ + { + "reference": "urn:uuid:1d000ca8-1d46-4c7e-a5a1-4dede883d961", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f1a78a45-429a-41cd-8416-106d9948e2ba", + "resource": { + "resourceType": "Claim", + "id": "f1a78a45-429a-41cd-8416-106d9948e2ba", + "type": "institutional", + "organization": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 454.36, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e", + "resource": { + "resourceType": "Encounter", + "id": "a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2016-06-17T12:47:52-07:00", + "end": "2016-06-17T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:262360cb-9fc9-496c-aa0b-b81d8ae62060", + "resource": { + "resourceType": "MedicationOrder", + "id": "262360cb-9fc9-496c-aa0b-b81d8ae62060", + "dateWritten": "2016-06-17T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "757594", + "display": "Jolivette 28 Day Pack" + } + ], + "text": "Jolivette 28 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:b3639899-7c53-423d-86ed-352c8f5af43c", + "resource": { + "resourceType": "Claim", + "id": "b3639899-7c53-423d-86ed-352c8f5af43c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:262360cb-9fc9-496c-aa0b-b81d8ae62060" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2430fd45-8ca8-4d54-b6fb-76b5448d0e2c", + "resource": { + "resourceType": "Claim", + "id": "2430fd45-8ca8-4d54-b6fb-76b5448d0e2c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048", + "resource": { + "resourceType": "Encounter", + "id": "9e720654-ab2d-4c64-a0c5-92912f21d048", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2016-08-02T12:47:52-07:00", + "end": "2016-08-02T13:02:52-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5798b5d3-df47-4eb8-9ad5-aea32bfce5b0", + "resource": { + "resourceType": "Condition", + "id": "5798b5d3-df47-4eb8-9ad5-aea32bfce5b0", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048" + }, + "dateRecorded": "2016-08-02", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2016-08-02T12:47:52-07:00", + "abatementDateTime": "2016-08-11T12:47:52-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:496feb8f-6322-43c2-90e0-adc05a760ccc", + "resource": { + "resourceType": "Observation", + "id": "496feb8f-6322-43c2-90e0-adc05a760ccc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048" + }, + "effectiveDateTime": "2016-08-02T12:47:52-07:00", + "issued": "2016-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 37.502, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92a34b1c-fd71-4f23-aaa5-304f49968548", + "resource": { + "resourceType": "Claim", + "id": "92a34b1c-fd71-4f23-aaa5-304f49968548", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:94372af5-4177-4757-991e-eeab8666cb4d", + "resource": { + "resourceType": "Encounter", + "id": "94372af5-4177-4757-991e-eeab8666cb4d", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2017-06-12T12:47:52-07:00", + "end": "2017-06-12T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d68f95cd-cdc6-4f3a-a9df-4c64355f0d62", + "resource": { + "resourceType": "MedicationOrder", + "id": "d68f95cd-cdc6-4f3a-a9df-4c64355f0d62", + "dateWritten": "2017-06-12T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:94372af5-4177-4757-991e-eeab8666cb4d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "751905", + "display": "Trinessa 28 Day Pack" + } + ], + "text": "Trinessa 28 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:74728217-e9d5-476e-9cd4-50dbcdc23b29", + "resource": { + "resourceType": "Claim", + "id": "74728217-e9d5-476e-9cd4-50dbcdc23b29", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:d68f95cd-cdc6-4f3a-a9df-4c64355f0d62" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:96d694e8-1878-4096-a912-5d5262c9919a", + "resource": { + "resourceType": "Claim", + "id": "96d694e8-1878-4096-a912-5d5262c9919a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762", + "resource": { + "resourceType": "Encounter", + "id": "ed971cc5-27c9-41d4-a5ef-f1d22a097762", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" + } + } + ], + "period": { + "start": "2017-07-28T12:47:52-07:00", + "end": "2017-07-28T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:513096da-0208-41fb-a191-92d61ec98d9d", + "resource": { + "resourceType": "Observation", + "id": "513096da-0208-41fb-a191-92d61ec98d9d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 162.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:24b3d8a6-fd2d-4c9f-9ee4-3b0faf9d94f8", + "resource": { + "resourceType": "Observation", + "id": "24b3d8a6-fd2d-4c9f-9ee4-3b0faf9d94f8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36d4bbd6-a005-4c3d-bc90-90b5bef3859b", + "resource": { + "resourceType": "Observation", + "id": "36d4bbd6-a005-4c3d-bc90-90b5bef3859b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 72.700, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3c7de8e-7577-483c-a5e3-b0a4547f646a", + "resource": { + "resourceType": "Observation", + "id": "b3c7de8e-7577-483c-a5e3-b0a4547f646a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 27.430, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:066f2d74-e6c8-423b-ba18-daf2bc55962f", + "resource": { + "resourceType": "Observation", + "id": "066f2d74-e6c8-423b-ba18-daf2bc55962f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68245bf5-1481-4a79-92b1-f37c4854ff06", + "resource": { + "resourceType": "Observation", + "id": "68245bf5-1481-4a79-92b1-f37c4854ff06", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 98, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59458042-1e08-4baa-b57b-0da4f0999cd9", + "resource": { + "resourceType": "Observation", + "id": "59458042-1e08-4baa-b57b-0da4f0999cd9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cb4838e-a1ce-4d2d-b7b7-a3e1510ac22b", + "resource": { + "resourceType": "Observation", + "id": "0cb4838e-a1ce-4d2d-b7b7-a3e1510ac22b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 187.95, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8bc571a6-8900-43f4-9de6-16e9e344c153", + "resource": { + "resourceType": "Observation", + "id": "8bc571a6-8900-43f4-9de6-16e9e344c153", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 148.96, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f10ce4b4-57db-4aa8-b9a3-1f475d97a463", + "resource": { + "resourceType": "Observation", + "id": "f10ce4b4-57db-4aa8-b9a3-1f475d97a463", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 89.710, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c74ce653-d7ec-4d3c-bead-2cac242570d1", + "resource": { + "resourceType": "Observation", + "id": "c74ce653-d7ec-4d3c-bead-2cac242570d1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueQuantity": { + "value": 68.450, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d983d75-de81-4b9c-9aa4-a4c02c36afdd", + "resource": { + "resourceType": "Observation", + "id": "9d983d75-de81-4b9c-9aa4-a4c02c36afdd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ccbfc84-232a-4033-b136-2c5ff11b6a7f", + "resource": { + "resourceType": "Immunization", + "id": "9ccbfc84-232a-4033-b136-2c5ff11b6a7f", + "status": "completed", + "date": "2017-07-28T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ac725ef6-5617-4b54-872c-7b2b0c6e8827", + "resource": { + "resourceType": "Immunization", + "id": "ac725ef6-5617-4b54-872c-7b2b0c6e8827", + "status": "completed", + "date": "2017-07-28T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f743fe11-6db2-4b06-8c84-e26401cbfe5c", + "resource": { + "resourceType": "Immunization", + "id": "f743fe11-6db2-4b06-8c84-e26401cbfe5c", + "status": "completed", + "date": "2017-07-28T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + } + ], + "text": "Hep A, adult" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:79bfa9f5-8a60-4296-8969-1ed167e5f550", + "resource": { + "resourceType": "DiagnosticReport", + "id": "79bfa9f5-8a60-4296-8969-1ed167e5f550", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" + }, + "effectiveDateTime": "2017-07-28T12:47:52-07:00", + "issued": "2017-07-28T12:47:52.730-07:00", + "performer": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "result": [ + { + "reference": "urn:uuid:c74ce653-d7ec-4d3c-bead-2cac242570d1", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:c2823519-9690-4b77-9f15-cbecf0fd2a6d", + "resource": { + "resourceType": "Claim", + "id": "c2823519-9690-4b77-9f15-cbecf0fd2a6d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8b4c29d4-d667-450c-96d8-bb3b04255396", + "resource": { + "resourceType": "Encounter", + "id": "8b4c29d4-d667-450c-96d8-bb3b04255396", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2018-06-07T12:47:52-07:00", + "end": "2018-06-07T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bc75a463-d467-4698-afa1-d22b5aeaff60", + "resource": { + "resourceType": "MedicationOrder", + "id": "bc75a463-d467-4698-afa1-d22b5aeaff60", + "dateWritten": "2018-06-07T12:47:52-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:8b4c29d4-d667-450c-96d8-bb3b04255396" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "978950", + "display": "Natazia 28 Day Pack" + } + ], + "text": "Natazia 28 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:7c8893ed-2611-41eb-bf80-6fe6b7b992c8", + "resource": { + "resourceType": "Claim", + "id": "7c8893ed-2611-41eb-bf80-6fe6b7b992c8", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:bc75a463-d467-4698-afa1-d22b5aeaff60" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c625cf6f-ad18-49ae-ac3b-ef257f0f3f1c", + "resource": { + "resourceType": "Claim", + "id": "c625cf6f-ad18-49ae-ac3b-ef257f0f3f1c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a44a3669-8d9e-4e72-8733-f85346a8688d", + "resource": { + "resourceType": "Encounter", + "id": "a44a3669-8d9e-4e72-8733-f85346a8688d", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + } + } + ], + "period": { + "start": "2019-06-02T12:47:52-07:00", + "end": "2019-06-02T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:044faa36-ce9d-4c99-9de1-f629f43d4d01", + "resource": { + "resourceType": "MedicationOrder", + "id": "044faa36-ce9d-4c99-9de1-f629f43d4d01", + "dateWritten": "2019-06-02T12:47:52-07:00", + "status": "active", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "prescriber": { + "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" + }, + "encounter": { + "reference": "urn:uuid:a44a3669-8d9e-4e72-8733-f85346a8688d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "749762", + "display": "Seasonique 91 Day Pack" + } + ], + "text": "Seasonique 91 Day Pack" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:3283f083-75c4-45f6-858a-718e373d599b", + "resource": { + "resourceType": "Claim", + "id": "3283f083-75c4-45f6-858a-718e373d599b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:044faa36-ce9d-4c99-9de1-f629f43d4d01" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:608ba58c-06e0-4426-86ce-08f4ec7815fa", + "resource": { + "resourceType": "Claim", + "id": "608ba58c-06e0-4426-86ce-08f4ec7815fa", + "type": "institutional", + "organization": { + "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c", + "resource": { + "resourceType": "Encounter", + "id": "ac8f9312-f86a-47b6-9af1-2359e349d65c", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" + } + } + ], + "period": { + "start": "2019-08-02T12:47:52-07:00", + "end": "2019-08-02T13:02:52-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5f02d23a-f841-474b-b1a0-5e4b01e73ebd", + "resource": { + "resourceType": "Observation", + "id": "5f02d23a-f841-474b-b1a0-5e4b01e73ebd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 162.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aff5cc5f-74d9-4347-ad17-7025ed997578", + "resource": { + "resourceType": "Observation", + "id": "aff5cc5f-74d9-4347-ad17-7025ed997578", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11c3c9c0-d15b-46b5-8f35-ff534134ac1a", + "resource": { + "resourceType": "Observation", + "id": "11c3c9c0-d15b-46b5-8f35-ff534134ac1a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 76.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c94e846-884d-45f5-bbb7-21fa3c4ea842", + "resource": { + "resourceType": "Observation", + "id": "9c94e846-884d-45f5-bbb7-21fa3c4ea842", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 28.730, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b732f3de-8ad0-4f30-9080-a42b1a80948b", + "resource": { + "resourceType": "Observation", + "id": "b732f3de-8ad0-4f30-9080-a42b1a80948b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 132, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f9e36a87-9b2b-40c1-b818-371411feb397", + "resource": { + "resourceType": "Observation", + "id": "f9e36a87-9b2b-40c1-b818-371411feb397", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d6a70dc-c941-4414-9919-7d26319a9939", + "resource": { + "resourceType": "Observation", + "id": "0d6a70dc-c941-4414-9919-7d26319a9939", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:81a12a37-720e-4bee-8b81-ef7b7403c90a", + "resource": { + "resourceType": "Observation", + "id": "81a12a37-720e-4bee-8b81-ef7b7403c90a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + }, + "effectiveDateTime": "2019-08-02T12:47:52-07:00", + "issued": "2019-08-02T12:47:52.730-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82e2f210-8197-40b1-bef2-907f7d800975", + "resource": { + "resourceType": "Immunization", + "id": "82e2f210-8197-40b1-bef2-907f7d800975", + "status": "completed", + "date": "2019-08-02T12:47:52-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:653af184-26f4-47a6-adc2-1648928d6be4", + "resource": { + "resourceType": "Claim", + "id": "653af184-26f4-47a6-adc2-1648928d6be4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json new file mode 100644 index 000000000000..ac94ea8d8adc --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json @@ -0,0 +1,17399 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5", + "resource": { + "resourceType": "Patient", + "id": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 5597799561560712484 Population seed: 1586309754086
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Race", + "code": "2106-3", + "display": "White" + } + ], + "text": "White" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Ethnicity", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + ], + "text": "Not Hispanic or Latino" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Cindie288 Beer512" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "M" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Easton", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.20976015150670563 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 8.790239848493295 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR" + } + ] + }, + "system": "http://hospital.smarthealthit.org", + "value": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB" + } + ] + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-15-6629" + } + ], + "name": [ + { + "use": "official", + "family": [ + "Dietrich576" + ], + "given": [ + "Ernesto186" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-410-4513", + "use": "home" + } + ], + "gender": "male", + "birthDate": "2010-11-25", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.328966094980274 + }, + { + "url": "longitude", + "valueDecimal": -71.06791912415248 + } + ] + } + ], + "line": [ + "784 Johns Tunnel Apt 71" + ], + "city": "Boston", + "state": "Massachusetts", + "postalCode": "02110", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S" + } + ] + }, + "multipleBirthInteger": 2, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86", + "resource": { + "resourceType": "Organization", + "id": "70d8b93b-cc76-3c8b-8929-5aea213ecc86", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "BROOKLINE DERMATOLOGY ASSOCIATES, PC", + "telecom": [ + { + "system": "phone", + "value": "617-608-1575" + } + ], + "address": [ + { + "line": [ + "1208 B VFW PKWY" + ], + "city": "WEST ROXBURY", + "state": "MA", + "postalCode": "02132-4350", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e", + "resource": { + "resourceType": "Practitioner", + "id": "2d307854-d23a-337c-8215-e37f84fdc26e", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "66350" + } + ], + "active": true, + "name": { + "family": [ + "Eichmann909" + ], + "given": [ + "Marylou497" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "1208 B VFW PKWY" + ], + "city": "WEST ROXBURY", + "state": "MA", + "postalCode": "02132-4350", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58", + "resource": { + "resourceType": "Encounter", + "id": "2c1324ea-612c-4d84-849e-009205ea3b58", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2010-11-25T11:17:06-08:00", + "end": "2010-11-25T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8ebf2c45-def4-4c23-8e3f-c0e4ac79c959", + "resource": { + "resourceType": "Observation", + "id": "8ebf2c45-def4-4c23-8e3f-c0e4ac79c959", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 50.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fceaa3b5-ca1a-4e19-9e30-dba58f9e71e5", + "resource": { + "resourceType": "Observation", + "id": "fceaa3b5-ca1a-4e19-9e30-dba58f9e71e5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f9db9937-9f87-421a-a6b5-0cb08e75ebcd", + "resource": { + "resourceType": "Observation", + "id": "f9db9937-9f87-421a-a6b5-0cb08e75ebcd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 3.8000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:027003ed-8bd3-420f-9fe9-f0d9b521d2f2", + "resource": { + "resourceType": "Observation", + "id": "027003ed-8bd3-420f-9fe9-f0d9b521d2f2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 78.413, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d20b1c0-8d7e-42be-8f9d-0a258cb8fa39", + "resource": { + "resourceType": "Observation", + "id": "5d20b1c0-8d7e-42be-8f9d-0a258cb8fa39", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 33.860, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:16393f99-2b04-4a15-8d70-404521828062", + "resource": { + "resourceType": "Observation", + "id": "16393f99-2b04-4a15-8d70-404521828062", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 116, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33447bb9-8c10-430f-836e-fee7cf00820b", + "resource": { + "resourceType": "Observation", + "id": "33447bb9-8c10-430f-836e-fee7cf00820b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 95, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:20cf6011-db84-412c-b831-391592133d0d", + "resource": { + "resourceType": "Observation", + "id": "20cf6011-db84-412c-b831-391592133d0d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50934ad5-66ec-463a-a1c5-0bb9eb23a310", + "resource": { + "resourceType": "Observation", + "id": "50934ad5-66ec-463a-a1c5-0bb9eb23a310", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 6.4777, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4ab365d2-296f-4ac8-9e5a-ac816659a12e", + "resource": { + "resourceType": "Observation", + "id": "4ab365d2-296f-4ac8-9e5a-ac816659a12e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 4.0205, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d5efd4a1-76aa-4a08-ba76-99de85d0b502", + "resource": { + "resourceType": "Observation", + "id": "d5efd4a1-76aa-4a08-ba76-99de85d0b502", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 13.276, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9e09db52-230b-4bfe-b0aa-ad5b073d2711", + "resource": { + "resourceType": "Observation", + "id": "9e09db52-230b-4bfe-b0aa-ad5b073d2711", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 37.735, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:63c1447e-6a0a-4954-a858-c703124b87a5", + "resource": { + "resourceType": "Observation", + "id": "63c1447e-6a0a-4954-a858-c703124b87a5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 89.968, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99c12063-3565-48b2-8558-99f45b9a7973", + "resource": { + "resourceType": "Observation", + "id": "99c12063-3565-48b2-8558-99f45b9a7973", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 32.199, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c8c0590-e592-427f-adb6-95440513b9ea", + "resource": { + "resourceType": "Observation", + "id": "4c8c0590-e592-427f-adb6-95440513b9ea", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 34.686, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0ac6566-41b0-401f-958d-abfe390ca4b0", + "resource": { + "resourceType": "Observation", + "id": "c0ac6566-41b0-401f-958d-abfe390ca4b0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 41.069, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0a165cc6-dad1-4f9e-b439-278351ab9138", + "resource": { + "resourceType": "Observation", + "id": "0a165cc6-dad1-4f9e-b439-278351ab9138", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 199.09, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7ac73c3-3dd0-4432-998f-ff713ac3e537", + "resource": { + "resourceType": "Observation", + "id": "a7ac73c3-3dd0-4432-998f-ff713ac3e537", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 388.77, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:121bb5d6-d2ae-428e-8209-e5eb73f2152f", + "resource": { + "resourceType": "Observation", + "id": "121bb5d6-d2ae-428e-8209-e5eb73f2152f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueQuantity": { + "value": 9.6619, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8921de55-87a0-4e6e-bce4-1dee4371a66f", + "resource": { + "resourceType": "Observation", + "id": "8921de55-87a0-4e6e-bce4-1dee4371a66f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65cc1f49-4886-4a84-bd17-1401b24835ce", + "resource": { + "resourceType": "Procedure", + "id": "65cc1f49-4886-4a84-bd17-1401b24835ce", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2010-11-25T11:17:06-08:00", + "end": "2010-11-25T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c6219428-432d-4ccf-88a1-239d079e786a", + "resource": { + "resourceType": "Immunization", + "id": "c6219428-432d-4ccf-88a1-239d079e786a", + "status": "completed", + "date": "2010-11-25T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:33f4287c-c55e-472f-8a6d-2d924be47b39", + "resource": { + "resourceType": "DiagnosticReport", + "id": "33f4287c-c55e-472f-8a6d-2d924be47b39", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" + }, + "effectiveDateTime": "2010-11-25T11:17:06-08:00", + "issued": "2010-11-25T11:17:06.181-08:00", + "performer": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "result": [ + { + "reference": "urn:uuid:121bb5d6-d2ae-428e-8209-e5eb73f2152f", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:758d41bd-402c-4fb7-b8a0-d0a2e3899b15", + "resource": { + "resourceType": "Claim", + "id": "758d41bd-402c-4fb7-b8a0-d0a2e3899b15", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 764.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c", + "resource": { + "resourceType": "Encounter", + "id": "9f97ee46-dcd4-4c81-a02b-fee0eff9f13c", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2010-12-30T11:17:06-08:00", + "end": "2010-12-30T11:32:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:17830cab-4bf5-48e0-8daa-baceb286132a", + "resource": { + "resourceType": "Observation", + "id": "17830cab-4bf5-48e0-8daa-baceb286132a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 54.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7aeff5d2-982c-4053-adc3-cff39eb60f20", + "resource": { + "resourceType": "Observation", + "id": "7aeff5d2-982c-4053-adc3-cff39eb60f20", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bab19a92-d92c-4ecc-b56b-bc7d8f7bc65c", + "resource": { + "resourceType": "Observation", + "id": "bab19a92-d92c-4ecc-b56b-bc7d8f7bc65c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 4.7000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fe2478d5-2312-4058-881a-5c3c8fca88b3", + "resource": { + "resourceType": "Observation", + "id": "fe2478d5-2312-4058-881a-5c3c8fca88b3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 69.568, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5665c9e0-2d96-461c-98bb-a93e7a5b4e66", + "resource": { + "resourceType": "Observation", + "id": "5665c9e0-2d96-461c-98bb-a93e7a5b4e66", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 37.580, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:00091da9-8395-40b3-8e0b-53f6bd0c2ec7", + "resource": { + "resourceType": "Observation", + "id": "00091da9-8395-40b3-8e0b-53f6bd0c2ec7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 133, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35c6de82-2533-4b1b-9c3c-64062b2263d7", + "resource": { + "resourceType": "Observation", + "id": "35c6de82-2533-4b1b-9c3c-64062b2263d7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 67, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:708c78c2-9ab4-4c76-b2e2-b0cd4e8b0a49", + "resource": { + "resourceType": "Observation", + "id": "708c78c2-9ab4-4c76-b2e2-b0cd4e8b0a49", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cea9b66e-93f5-4a92-8f75-6196465ac1fa", + "resource": { + "resourceType": "Observation", + "id": "cea9b66e-93f5-4a92-8f75-6196465ac1fa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + }, + "effectiveDateTime": "2010-12-30T11:17:06-08:00", + "issued": "2010-12-30T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9582a8d2-5838-4ab2-bf58-ea4ca0326490", + "resource": { + "resourceType": "Immunization", + "id": "9582a8d2-5838-4ab2-bf58-ea4ca0326490", + "status": "completed", + "date": "2010-12-30T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:9ea34bc3-2591-4771-acc8-9458194f5471", + "resource": { + "resourceType": "Claim", + "id": "9ea34bc3-2591-4771-acc8-9458194f5471", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4", + "resource": { + "resourceType": "Encounter", + "id": "98699fd9-2ee9-4308-8b5b-09d9d78bd8d4", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2011-03-03T11:17:06-08:00", + "end": "2011-03-03T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cfa21e58-5812-49d9-9e5b-c84795c54fa9", + "resource": { + "resourceType": "Observation", + "id": "cfa21e58-5812-49d9-9e5b-c84795c54fa9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 59.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3813e00-9ad7-421c-928c-2bfa8b403d50", + "resource": { + "resourceType": "Observation", + "id": "f3813e00-9ad7-421c-928c-2bfa8b403d50", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a98bbb5d-7d71-44de-9f4d-07e06e3735af", + "resource": { + "resourceType": "Observation", + "id": "a98bbb5d-7d71-44de-9f4d-07e06e3735af", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 6.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28f83f6a-0a0d-4591-b488-9fd8ed7c4b92", + "resource": { + "resourceType": "Observation", + "id": "28f83f6a-0a0d-4591-b488-9fd8ed7c4b92", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 63.292, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:63e63183-9097-409a-bfce-b8375d2eead8", + "resource": { + "resourceType": "Observation", + "id": "63e63183-9097-409a-bfce-b8375d2eead8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 40.360, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:523aa424-944a-46b5-b84c-f4d13375d373", + "resource": { + "resourceType": "Observation", + "id": "523aa424-944a-46b5-b84c-f4d13375d373", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 133, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2479917-579c-4450-b2b2-5ec99295e050", + "resource": { + "resourceType": "Observation", + "id": "e2479917-579c-4450-b2b2-5ec99295e050", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 77, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7143f445-203a-4523-a367-23986099d586", + "resource": { + "resourceType": "Observation", + "id": "7143f445-203a-4523-a367-23986099d586", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2f0dffcc-2efe-4020-85d8-200f43a1dbc2", + "resource": { + "resourceType": "Observation", + "id": "2f0dffcc-2efe-4020-85d8-200f43a1dbc2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + }, + "effectiveDateTime": "2011-03-03T11:17:06-08:00", + "issued": "2011-03-03T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c1eaee3b-865f-4415-b9c2-5afe3b0f5e94", + "resource": { + "resourceType": "Procedure", + "id": "c1eaee3b-865f-4415-b9c2-5afe3b0f5e94", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2011-03-03T11:17:06-08:00", + "end": "2011-03-03T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9709808b-2050-4721-b5f2-8b7815911f59", + "resource": { + "resourceType": "Immunization", + "id": "9709808b-2050-4721-b5f2-8b7815911f59", + "status": "completed", + "date": "2011-03-03T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b0e4acac-3741-41dd-8c50-22e7e5bb3f89", + "resource": { + "resourceType": "Immunization", + "id": "b0e4acac-3741-41dd-8c50-22e7e5bb3f89", + "status": "completed", + "date": "2011-03-03T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:dd8562b5-6a35-470f-8276-06e35f0033be", + "resource": { + "resourceType": "Immunization", + "id": "dd8562b5-6a35-470f-8276-06e35f0033be", + "status": "completed", + "date": "2011-03-03T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:da335d7d-824c-43d7-b4e6-9310c8575e03", + "resource": { + "resourceType": "Immunization", + "id": "da335d7d-824c-43d7-b4e6-9310c8575e03", + "status": "completed", + "date": "2011-03-03T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:90a87f3a-ef6e-490c-a8e0-773b61238422", + "resource": { + "resourceType": "Immunization", + "id": "90a87f3a-ef6e-490c-a8e0-773b61238422", + "status": "completed", + "date": "2011-03-03T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b9e4e4e6-26fd-4dbb-862e-332accb0a87f", + "resource": { + "resourceType": "Claim", + "id": "b9e4e4e6-26fd-4dbb-862e-332accb0a87f", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 603.15, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd", + "resource": { + "resourceType": "Encounter", + "id": "31109686-bf0b-410d-9920-1348ec1098fd", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2011-05-05T12:17:06-07:00", + "end": "2011-05-05T12:32:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:570d6b42-b71c-4499-9245-330fa0c5344e", + "resource": { + "resourceType": "Observation", + "id": "570d6b42-b71c-4499-9245-330fa0c5344e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 63.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a105f262-1c6b-4465-862d-f3d496f2e288", + "resource": { + "resourceType": "Observation", + "id": "a105f262-1c6b-4465-862d-f3d496f2e288", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d45089d-c984-42dc-8ac9-33951ec01bac", + "resource": { + "resourceType": "Observation", + "id": "7d45089d-c984-42dc-8ac9-33951ec01bac", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 7.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6af14676-fdce-4728-8705-4aea01e9abef", + "resource": { + "resourceType": "Observation", + "id": "6af14676-fdce-4728-8705-4aea01e9abef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 71.421, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4da7d003-c99a-4830-95df-ff04bd84f3d1", + "resource": { + "resourceType": "Observation", + "id": "4da7d003-c99a-4830-95df-ff04bd84f3d1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 42.110, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7ad1aae-d601-423b-bdbc-6e9207fbfceb", + "resource": { + "resourceType": "Observation", + "id": "a7ad1aae-d601-423b-bdbc-6e9207fbfceb", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbf20ae1-e0b1-48fa-833c-374255ba2fbe", + "resource": { + "resourceType": "Observation", + "id": "bbf20ae1-e0b1-48fa-833c-374255ba2fbe", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 95, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebaca6b5-0ced-4f46-9cb9-7fc4d310de83", + "resource": { + "resourceType": "Observation", + "id": "ebaca6b5-0ced-4f46-9cb9-7fc4d310de83", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8dbcb168-e08d-403c-972d-aac59eaafcc1", + "resource": { + "resourceType": "Observation", + "id": "8dbcb168-e08d-403c-972d-aac59eaafcc1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + }, + "effectiveDateTime": "2011-05-05T12:17:06-07:00", + "issued": "2011-05-05T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d7f7d12f-8064-4a9e-96cd-7648afab0563", + "resource": { + "resourceType": "Immunization", + "id": "d7f7d12f-8064-4a9e-96cd-7648afab0563", + "status": "completed", + "date": "2011-05-05T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0211cf39-777c-4ff4-8100-06486c9f10c6", + "resource": { + "resourceType": "Immunization", + "id": "0211cf39-777c-4ff4-8100-06486c9f10c6", + "status": "completed", + "date": "2011-05-05T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b794328e-9e13-4c6a-9d28-de8699523a71", + "resource": { + "resourceType": "Immunization", + "id": "b794328e-9e13-4c6a-9d28-de8699523a71", + "status": "completed", + "date": "2011-05-05T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2f2e3ad1-b57b-45d2-8f0b-cab25ca48fb7", + "resource": { + "resourceType": "Immunization", + "id": "2f2e3ad1-b57b-45d2-8f0b-cab25ca48fb7", + "status": "completed", + "date": "2011-05-05T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ead072ed-047e-45af-9100-0ac8f1dea0a3", + "resource": { + "resourceType": "Immunization", + "id": "ead072ed-047e-45af-9100-0ac8f1dea0a3", + "status": "completed", + "date": "2011-05-05T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e69956de-942c-4392-8ec3-b5e739d40d31", + "resource": { + "resourceType": "Claim", + "id": "e69956de-942c-4392-8ec3-b5e739d40d31", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc", + "resource": { + "resourceType": "Encounter", + "id": "2745c21c-b71f-4bba-9483-c3f5a54cd4fc", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2011-08-04T12:17:06-07:00", + "end": "2011-08-04T12:47:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:de78941e-6fe9-41c4-b476-e43c6b1e20eb", + "resource": { + "resourceType": "Condition", + "id": "de78941e-6fe9-41c4-b476-e43c6b1e20eb", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "dateRecorded": "2011-08-04", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53741008", + "display": "Coronary Heart Disease" + } + ], + "text": "Coronary Heart Disease" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2011-08-04T12:17:06-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b1becaba-940c-4bee-a906-80a7a507576f", + "resource": { + "resourceType": "Observation", + "id": "b1becaba-940c-4bee-a906-80a7a507576f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 68.400, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:840de12e-a048-4777-ac9c-c4936d7a116a", + "resource": { + "resourceType": "Observation", + "id": "840de12e-a048-4777-ac9c-c4936d7a116a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e867d25e-924d-421e-bd24-582a42999c9c", + "resource": { + "resourceType": "Observation", + "id": "e867d25e-924d-421e-bd24-582a42999c9c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 8.8000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:880dcbf4-d70a-4d8e-9e09-8ff3ba08228f", + "resource": { + "resourceType": "Observation", + "id": "880dcbf4-d70a-4d8e-9e09-8ff3ba08228f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 84.866, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:03e0b34c-edc1-4c7c-9649-c92b1aea0751", + "resource": { + "resourceType": "Observation", + "id": "03e0b34c-edc1-4c7c-9649-c92b1aea0751", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 43.830, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:62f0376a-f867-4808-b449-ecec3d0cb5fd", + "resource": { + "resourceType": "Observation", + "id": "62f0376a-f867-4808-b449-ecec3d0cb5fd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbb170a3-0b9a-4ec4-9d08-b9877ce5c654", + "resource": { + "resourceType": "Observation", + "id": "bbb170a3-0b9a-4ec4-9d08-b9877ce5c654", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 94, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f8701862-10d7-4ba9-8e86-9fdf26d0d736", + "resource": { + "resourceType": "Observation", + "id": "f8701862-10d7-4ba9-8e86-9fdf26d0d736", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c0e9599-cf99-4fe6-9e04-2da95998ab83", + "resource": { + "resourceType": "Observation", + "id": "5c0e9599-cf99-4fe6-9e04-2da95998ab83", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "effectiveDateTime": "2011-08-04T12:17:06-07:00", + "issued": "2011-08-04T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dee0f682-2456-4954-a845-a4a42c043469", + "resource": { + "resourceType": "Procedure", + "id": "dee0f682-2456-4954-a845-a4a42c043469", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2011-08-04T12:17:06-07:00", + "end": "2011-08-04T12:32:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a0d6efed-0215-4fcf-bafe-1580e606c135", + "resource": { + "resourceType": "MedicationOrder", + "id": "a0d6efed-0215-4fcf-bafe-1580e606c135", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:6b4dc1e0-7cbe-40e4-952c-37fc24919f1c", + "resource": { + "resourceType": "Claim", + "id": "6b4dc1e0-7cbe-40e4-952c-37fc24919f1c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a0d6efed-0215-4fcf-bafe-1580e606c135" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4559ce59-f77f-4935-9f38-7237e231a096", + "resource": { + "resourceType": "MedicationOrder", + "id": "4559ce59-f77f-4935-9f38-7237e231a096", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:2be067d2-2d80-4f98-b4e9-e7e70bc9d04a", + "resource": { + "resourceType": "Claim", + "id": "2be067d2-2d80-4f98-b4e9-e7e70bc9d04a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:4559ce59-f77f-4935-9f38-7237e231a096" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b85c572e-5bfa-427d-95db-4b588c360d10", + "resource": { + "resourceType": "MedicationOrder", + "id": "b85c572e-5bfa-427d-95db-4b588c360d10", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:a3eaaf3e-3b6b-4a1a-a726-c38ede34a454", + "resource": { + "resourceType": "Claim", + "id": "a3eaaf3e-3b6b-4a1a-a726-c38ede34a454", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:b85c572e-5bfa-427d-95db-4b588c360d10" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:55dd11b1-20d5-476a-a492-eacd0a925069", + "resource": { + "resourceType": "MedicationOrder", + "id": "55dd11b1-20d5-476a-a492-eacd0a925069", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:132079e0-2aa9-4267-855e-62a9127ed284", + "resource": { + "resourceType": "Claim", + "id": "132079e0-2aa9-4267-855e-62a9127ed284", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:55dd11b1-20d5-476a-a492-eacd0a925069" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:92eb4cd0-be66-4476-acc6-0b88a7624555", + "resource": { + "resourceType": "MedicationOrder", + "id": "92eb4cd0-be66-4476-acc6-0b88a7624555", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:cd469123-20ff-4ce0-9382-69aa82ae7f52", + "resource": { + "resourceType": "Claim", + "id": "cd469123-20ff-4ce0-9382-69aa82ae7f52", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:92eb4cd0-be66-4476-acc6-0b88a7624555" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ac2d62f3-484f-4762-96ff-a1bf61bafbf5", + "resource": { + "resourceType": "MedicationOrder", + "id": "ac2d62f3-484f-4762-96ff-a1bf61bafbf5", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:23cec18d-f254-4d89-80d3-12c19aaabeca", + "resource": { + "resourceType": "Claim", + "id": "23cec18d-f254-4d89-80d3-12c19aaabeca", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:ac2d62f3-484f-4762-96ff-a1bf61bafbf5" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:053ac539-d5c1-4873-b76c-f6258ee3d6aa", + "resource": { + "resourceType": "MedicationOrder", + "id": "053ac539-d5c1-4873-b76c-f6258ee3d6aa", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:14159c4f-9943-4128-a1cf-ea08b1d7713e", + "resource": { + "resourceType": "Claim", + "id": "14159c4f-9943-4128-a1cf-ea08b1d7713e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:053ac539-d5c1-4873-b76c-f6258ee3d6aa" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:16f34136-9f5a-412d-88e0-a63f927b7307", + "resource": { + "resourceType": "MedicationOrder", + "id": "16f34136-9f5a-412d-88e0-a63f927b7307", + "dateWritten": "2011-08-04T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:79ee24b2-23ee-4ee4-af1d-a3d5b592e129", + "resource": { + "resourceType": "Claim", + "id": "79ee24b2-23ee-4ee4-af1d-a3d5b592e129", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:16f34136-9f5a-412d-88e0-a63f927b7307" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bf4f03f9-4995-4d57-9bad-c68805e10e56", + "resource": { + "resourceType": "Immunization", + "id": "bf4f03f9-4995-4d57-9bad-c68805e10e56", + "status": "completed", + "date": "2011-08-04T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:214e3558-1979-4783-98b0-3325539a3d6c", + "resource": { + "resourceType": "Immunization", + "id": "214e3558-1979-4783-98b0-3325539a3d6c", + "status": "completed", + "date": "2011-08-04T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:9a0f2346-7a9f-4857-9bbf-88e1d8c8e8e3", + "resource": { + "resourceType": "Immunization", + "id": "9a0f2346-7a9f-4857-9bbf-88e1d8c8e8e3", + "status": "completed", + "date": "2011-08-04T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2006e537-92e8-4850-9cee-a26da05d7b98", + "resource": { + "resourceType": "Immunization", + "id": "2006e537-92e8-4850-9cee-a26da05d7b98", + "status": "completed", + "date": "2011-08-04T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8dbba982-dd89-4979-a288-5e9161502b1e", + "resource": { + "resourceType": "Immunization", + "id": "8dbba982-dd89-4979-a288-5e9161502b1e", + "status": "completed", + "date": "2011-08-04T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:39f0d03a-9e58-49c6-a7a3-470ccbe8c1cd", + "resource": { + "resourceType": "Claim", + "id": "39f0d03a-9e58-49c6-a7a3-470ccbe8c1cd", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "53741008", + "display": "Coronary Heart Disease" + } + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 8, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 699.53, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287", + "resource": { + "resourceType": "Encounter", + "id": "0d6730ed-1bc4-40ca-9483-9b46ce6f1287", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2011-11-03T12:17:06-07:00", + "end": "2011-11-03T12:32:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9a8ab457-866c-4253-82f3-895d6cb0642d", + "resource": { + "resourceType": "Observation", + "id": "9a8ab457-866c-4253-82f3-895d6cb0642d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 72.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0f54d30e-add4-4071-a554-bff23ea00150", + "resource": { + "resourceType": "Observation", + "id": "0f54d30e-add4-4071-a554-bff23ea00150", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35ff6076-2462-48a0-a8b0-be407d1fdb9a", + "resource": { + "resourceType": "Observation", + "id": "35ff6076-2462-48a0-a8b0-be407d1fdb9a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 9.9000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecc38ff1-69be-4d09-bdf6-201aa0ad43ae", + "resource": { + "resourceType": "Observation", + "id": "ecc38ff1-69be-4d09-bdf6-201aa0ad43ae", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 86.451, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2962ef3f-d9d0-4b4c-a89e-f2f27b7daeb8", + "resource": { + "resourceType": "Observation", + "id": "2962ef3f-d9d0-4b4c-a89e-f2f27b7daeb8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 44.980, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e75e6156-ee15-4c17-ace7-efd2328f3066", + "resource": { + "resourceType": "Observation", + "id": "e75e6156-ee15-4c17-ace7-efd2328f3066", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12d4913a-9e64-44ff-87b3-16fbf9c9de94", + "resource": { + "resourceType": "Observation", + "id": "12d4913a-9e64-44ff-87b3-16fbf9c9de94", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 80, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b5765ee-4a0f-4dea-8c74-ffd1bd14a474", + "resource": { + "resourceType": "Observation", + "id": "2b5765ee-4a0f-4dea-8c74-ffd1bd14a474", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a402273a-5722-468a-8c9b-4a7db2182a85", + "resource": { + "resourceType": "Observation", + "id": "a402273a-5722-468a-8c9b-4a7db2182a85", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "effectiveDateTime": "2011-11-03T12:17:06-07:00", + "issued": "2011-11-03T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e787d579-dc45-475b-bf2e-cac9ce29366e", + "resource": { + "resourceType": "MedicationOrder", + "id": "e787d579-dc45-475b-bf2e-cac9ce29366e", + "dateWritten": "2011-11-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:a08297f4-01b8-481c-96e2-2f61c11b7ec7", + "resource": { + "resourceType": "Claim", + "id": "a08297f4-01b8-481c-96e2-2f61c11b7ec7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:e787d579-dc45-475b-bf2e-cac9ce29366e" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9d8ebca3-9068-4768-b34a-a73776cbdad0", + "resource": { + "resourceType": "MedicationOrder", + "id": "9d8ebca3-9068-4768-b34a-a73776cbdad0", + "dateWritten": "2011-11-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:b69bb887-fefe-4cfe-a0ea-7a58fed29f68", + "resource": { + "resourceType": "Claim", + "id": "b69bb887-fefe-4cfe-a0ea-7a58fed29f68", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:9d8ebca3-9068-4768-b34a-a73776cbdad0" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4675571a-bfbe-42ef-b661-ee543a680571", + "resource": { + "resourceType": "MedicationOrder", + "id": "4675571a-bfbe-42ef-b661-ee543a680571", + "dateWritten": "2011-11-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:cf0be6be-3e04-463f-9a08-a56b2ef4b530", + "resource": { + "resourceType": "Claim", + "id": "cf0be6be-3e04-463f-9a08-a56b2ef4b530", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:4675571a-bfbe-42ef-b661-ee543a680571" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8556eace-d21f-4635-b8ef-e2820f5c70ce", + "resource": { + "resourceType": "MedicationOrder", + "id": "8556eace-d21f-4635-b8ef-e2820f5c70ce", + "dateWritten": "2011-11-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:65200abf-a80e-4ebc-9aa1-3a2a403826e9", + "resource": { + "resourceType": "Claim", + "id": "65200abf-a80e-4ebc-9aa1-3a2a403826e9", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:8556eace-d21f-4635-b8ef-e2820f5c70ce" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:36996e46-a7f8-4495-a000-2d1e284e2437", + "resource": { + "resourceType": "Claim", + "id": "36996e46-a7f8-4495-a000-2d1e284e2437", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3", + "resource": { + "resourceType": "Encounter", + "id": "7ff08b3d-8101-4e74-b528-f4709ad7bbe3", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2012-02-02T11:17:06-08:00", + "end": "2012-02-02T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:530cce89-b948-4d6c-95e1-bc8c22afcb1a", + "resource": { + "resourceType": "Observation", + "id": "530cce89-b948-4d6c-95e1-bc8c22afcb1a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 75.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:abdf4c7a-865f-4a02-853d-3c9274aa60e0", + "resource": { + "resourceType": "Observation", + "id": "abdf4c7a-865f-4a02-853d-3c9274aa60e0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b4957d1-01ac-4ed8-81ae-13af964d44d2", + "resource": { + "resourceType": "Observation", + "id": "6b4957d1-01ac-4ed8-81ae-13af964d44d2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 10.700, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6858eb67-2ced-4085-b704-a61c8b1d0925", + "resource": { + "resourceType": "Observation", + "id": "6858eb67-2ced-4085-b704-a61c8b1d0925", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 80.884, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff0a45c8-11c5-4f03-a277-304dc35a9c85", + "resource": { + "resourceType": "Observation", + "id": "ff0a45c8-11c5-4f03-a277-304dc35a9c85", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 45.810, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6c8fae00-80b4-4539-a01f-0ec48a6fae91", + "resource": { + "resourceType": "Observation", + "id": "6c8fae00-80b4-4539-a01f-0ec48a6fae91", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 71, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 107, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d97c313f-17ea-40b5-b4af-3328e1f9fa1a", + "resource": { + "resourceType": "Observation", + "id": "d97c313f-17ea-40b5-b4af-3328e1f9fa1a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 95, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d44864cf-98b0-42c4-a9cd-37d9b423f5cc", + "resource": { + "resourceType": "Observation", + "id": "d44864cf-98b0-42c4-a9cd-37d9b423f5cc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a77d622e-04a5-414c-9e07-075093068075", + "resource": { + "resourceType": "Observation", + "id": "a77d622e-04a5-414c-9e07-075093068075", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "effectiveDateTime": "2012-02-02T11:17:06-08:00", + "issued": "2012-02-02T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:392921a7-98f1-43df-be6d-ec45da239376", + "resource": { + "resourceType": "Procedure", + "id": "392921a7-98f1-43df-be6d-ec45da239376", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2012-02-02T11:17:06-08:00", + "end": "2012-02-02T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:204d30f2-01a4-4347-be9c-8f1cfb77b251", + "resource": { + "resourceType": "MedicationOrder", + "id": "204d30f2-01a4-4347-be9c-8f1cfb77b251", + "dateWritten": "2012-02-02T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:4f42779d-89c4-4ae4-bb4e-b040460aa4a2", + "resource": { + "resourceType": "Claim", + "id": "4f42779d-89c4-4ae4-bb4e-b040460aa4a2", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:204d30f2-01a4-4347-be9c-8f1cfb77b251" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:412738a7-1b3d-4727-b390-f24369bbdb88", + "resource": { + "resourceType": "MedicationOrder", + "id": "412738a7-1b3d-4727-b390-f24369bbdb88", + "dateWritten": "2012-02-02T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:1bb60d5d-bf69-4a63-b906-f01b74d8bb99", + "resource": { + "resourceType": "Claim", + "id": "1bb60d5d-bf69-4a63-b906-f01b74d8bb99", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:412738a7-1b3d-4727-b390-f24369bbdb88" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d69c2510-34be-495f-93da-3cb81620cf57", + "resource": { + "resourceType": "MedicationOrder", + "id": "d69c2510-34be-495f-93da-3cb81620cf57", + "dateWritten": "2012-02-02T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:ff381297-d1a8-4a39-865c-60b2f828177c", + "resource": { + "resourceType": "Claim", + "id": "ff381297-d1a8-4a39-865c-60b2f828177c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:d69c2510-34be-495f-93da-3cb81620cf57" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c16b38ad-e77e-4782-924c-e4e92d92b0a1", + "resource": { + "resourceType": "MedicationOrder", + "id": "c16b38ad-e77e-4782-924c-e4e92d92b0a1", + "dateWritten": "2012-02-02T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:7647de31-3286-4bf2-a74d-43aacdef3819", + "resource": { + "resourceType": "Claim", + "id": "7647de31-3286-4bf2-a74d-43aacdef3819", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:c16b38ad-e77e-4782-924c-e4e92d92b0a1" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ad6fb0e5-49d8-4653-a76d-bbf3457031bd", + "resource": { + "resourceType": "Immunization", + "id": "ad6fb0e5-49d8-4653-a76d-bbf3457031bd", + "status": "completed", + "date": "2012-02-02T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a513a582-d4c1-4212-a145-6ce2851137db", + "resource": { + "resourceType": "Immunization", + "id": "a513a582-d4c1-4212-a145-6ce2851137db", + "status": "completed", + "date": "2012-02-02T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f5a25186-8040-4bcf-a2c0-2f45e07b168e", + "resource": { + "resourceType": "Immunization", + "id": "f5a25186-8040-4bcf-a2c0-2f45e07b168e", + "status": "completed", + "date": "2012-02-02T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e1e69273-8ab2-4504-9f36-391d9486a71c", + "resource": { + "resourceType": "Immunization", + "id": "e1e69273-8ab2-4504-9f36-391d9486a71c", + "status": "completed", + "date": "2012-02-02T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2ba70002-3750-4fd3-904a-967cf3abb1fe", + "resource": { + "resourceType": "Immunization", + "id": "2ba70002-3750-4fd3-904a-967cf3abb1fe", + "status": "completed", + "date": "2012-02-02T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:49c50f34-3a7c-4577-832c-b971599dbe6e", + "resource": { + "resourceType": "Claim", + "id": "49c50f34-3a7c-4577-832c-b971599dbe6e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 460.25, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6", + "resource": { + "resourceType": "Encounter", + "id": "465b8554-145a-41bf-9423-dc26a4d195f6", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2012-05-03T12:17:06-07:00", + "end": "2012-05-03T12:32:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:53e25dfb-8a4b-47d5-9ddc-746fcc5862c8", + "resource": { + "resourceType": "Observation", + "id": "53e25dfb-8a4b-47d5-9ddc-746fcc5862c8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 78.400, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:da6acbbb-090d-460c-a2b9-04befb1928ef", + "resource": { + "resourceType": "Observation", + "id": "da6acbbb-090d-460c-a2b9-04befb1928ef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:56767a3d-01dd-4554-99c4-e1c4afe512d2", + "resource": { + "resourceType": "Observation", + "id": "56767a3d-01dd-4554-99c4-e1c4afe512d2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 11.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4451c32-7677-4954-a27d-578eb5a54262", + "resource": { + "resourceType": "Observation", + "id": "c4451c32-7677-4954-a27d-578eb5a54262", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 84.041, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3be791f-2004-4752-a42a-e58ed13f913b", + "resource": { + "resourceType": "Observation", + "id": "b3be791f-2004-4752-a42a-e58ed13f913b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 46.420, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2e7825de-e0f7-4d15-a6ee-03df392572e2", + "resource": { + "resourceType": "Observation", + "id": "2e7825de-e0f7-4d15-a6ee-03df392572e2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e5aa27b2-4c33-4047-974d-a70a917aa5d3", + "resource": { + "resourceType": "Observation", + "id": "e5aa27b2-4c33-4047-974d-a70a917aa5d3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a00c22c3-e18c-4ad4-973e-00daf5adae45", + "resource": { + "resourceType": "Observation", + "id": "a00c22c3-e18c-4ad4-973e-00daf5adae45", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4dd0f81e-254c-4d1a-b29f-b90e3cd78b10", + "resource": { + "resourceType": "Observation", + "id": "4dd0f81e-254c-4d1a-b29f-b90e3cd78b10", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "effectiveDateTime": "2012-05-03T12:17:06-07:00", + "issued": "2012-05-03T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84f455d6-c625-4ce2-8130-7a6e94ec2d14", + "resource": { + "resourceType": "MedicationOrder", + "id": "84f455d6-c625-4ce2-8130-7a6e94ec2d14", + "dateWritten": "2012-05-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:003b3b0b-ec0b-4962-8f68-e5af7754dd3a", + "resource": { + "resourceType": "Claim", + "id": "003b3b0b-ec0b-4962-8f68-e5af7754dd3a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:84f455d6-c625-4ce2-8130-7a6e94ec2d14" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:38e77308-0d4a-470f-9713-9492eff388af", + "resource": { + "resourceType": "MedicationOrder", + "id": "38e77308-0d4a-470f-9713-9492eff388af", + "dateWritten": "2012-05-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:32d3acf9-c8e2-401f-b9f9-f1f192e5f8ab", + "resource": { + "resourceType": "Claim", + "id": "32d3acf9-c8e2-401f-b9f9-f1f192e5f8ab", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:38e77308-0d4a-470f-9713-9492eff388af" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a7086955-dd8a-4d67-ab5c-2c21475e3d4d", + "resource": { + "resourceType": "MedicationOrder", + "id": "a7086955-dd8a-4d67-ab5c-2c21475e3d4d", + "dateWritten": "2012-05-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:ff1a3820-04c3-4b6e-ad11-67104b17fe43", + "resource": { + "resourceType": "Claim", + "id": "ff1a3820-04c3-4b6e-ad11-67104b17fe43", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a7086955-dd8a-4d67-ab5c-2c21475e3d4d" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:04c17a70-6744-459f-9d96-a0b4d608f50b", + "resource": { + "resourceType": "MedicationOrder", + "id": "04c17a70-6744-459f-9d96-a0b4d608f50b", + "dateWritten": "2012-05-03T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:8fcb9d18-e96f-4627-81f1-8ed9b32ad6ee", + "resource": { + "resourceType": "Claim", + "id": "8fcb9d18-e96f-4627-81f1-8ed9b32ad6ee", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:04c17a70-6744-459f-9d96-a0b4d608f50b" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c010f27b-50ee-4280-b90c-e26a1c9ce671", + "resource": { + "resourceType": "Immunization", + "id": "c010f27b-50ee-4280-b90c-e26a1c9ce671", + "status": "completed", + "date": "2012-05-03T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:28f8607b-3bab-478a-873b-31686e483fa0", + "resource": { + "resourceType": "Claim", + "id": "28f8607b-3bab-478a-873b-31686e483fa0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d", + "resource": { + "resourceType": "Encounter", + "id": "8a2ea952-1c47-4cf8-a748-d14ebfb5144d", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2012-11-01T12:17:06-07:00", + "end": "2012-11-01T12:47:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:40d0aeaf-90f0-4b5e-b8bc-540dafa6e58f", + "resource": { + "resourceType": "Observation", + "id": "40d0aeaf-90f0-4b5e-b8bc-540dafa6e58f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 83.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b66440b-c784-425d-911c-0a05df513978", + "resource": { + "resourceType": "Observation", + "id": "0b66440b-c784-425d-911c-0a05df513978", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df8e4146-01f1-4051-b074-300f8a6ec1a4", + "resource": { + "resourceType": "Observation", + "id": "df8e4146-01f1-4051-b074-300f8a6ec1a4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 12.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab3b7842-6fac-4402-872a-6911bc343fe3", + "resource": { + "resourceType": "Observation", + "id": "ab3b7842-6fac-4402-872a-6911bc343fe3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 67.104, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:901daf0f-cde0-46f3-8aea-37e9e57017f0", + "resource": { + "resourceType": "Observation", + "id": "901daf0f-cde0-46f3-8aea-37e9e57017f0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 47.25, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e69b070-974f-40f3-92b7-ab59cc947574", + "resource": { + "resourceType": "Observation", + "id": "4e69b070-974f-40f3-92b7-ab59cc947574", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d467c45-1e68-49d9-ae72-e57f296ccc31", + "resource": { + "resourceType": "Observation", + "id": "0d467c45-1e68-49d9-ae72-e57f296ccc31", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 99, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:07ba669b-ba3f-47e8-9933-77887f3a4bf8", + "resource": { + "resourceType": "Observation", + "id": "07ba669b-ba3f-47e8-9933-77887f3a4bf8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8087197c-1c20-4c51-bea7-2d6731d75bb4", + "resource": { + "resourceType": "Observation", + "id": "8087197c-1c20-4c51-bea7-2d6731d75bb4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "effectiveDateTime": "2012-11-01T12:17:06-07:00", + "issued": "2012-11-01T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ead65c5-4b51-4587-b3b9-b984ecd58370", + "resource": { + "resourceType": "Procedure", + "id": "0ead65c5-4b51-4587-b3b9-b984ecd58370", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2012-11-01T12:17:06-07:00", + "end": "2012-11-01T12:32:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3", + "resource": { + "resourceType": "MedicationOrder", + "id": "3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3", + "dateWritten": "2012-11-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:45dfb83d-2e0e-4c51-94c8-194abd11b1cf", + "resource": { + "resourceType": "Claim", + "id": "45dfb83d-2e0e-4c51-94c8-194abd11b1cf", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:af23f926-a8d4-4ae7-9dd3-f4f340e3066b", + "resource": { + "resourceType": "MedicationOrder", + "id": "af23f926-a8d4-4ae7-9dd3-f4f340e3066b", + "dateWritten": "2012-11-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:843584f6-fb45-4135-b3fe-829c961c418d", + "resource": { + "resourceType": "Claim", + "id": "843584f6-fb45-4135-b3fe-829c961c418d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:af23f926-a8d4-4ae7-9dd3-f4f340e3066b" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8b606c95-ef85-4dd4-82f6-dcb37e8c2b23", + "resource": { + "resourceType": "MedicationOrder", + "id": "8b606c95-ef85-4dd4-82f6-dcb37e8c2b23", + "dateWritten": "2012-11-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:48297181-250d-4925-ae64-715d3d3342ab", + "resource": { + "resourceType": "Claim", + "id": "48297181-250d-4925-ae64-715d3d3342ab", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:8b606c95-ef85-4dd4-82f6-dcb37e8c2b23" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a98058c1-7dbd-412c-bb9c-4c61123bcdb4", + "resource": { + "resourceType": "MedicationOrder", + "id": "a98058c1-7dbd-412c-bb9c-4c61123bcdb4", + "dateWritten": "2012-11-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:6d3ad287-cb02-4125-ad4d-3467e19e1afb", + "resource": { + "resourceType": "Claim", + "id": "6d3ad287-cb02-4125-ad4d-3467e19e1afb", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a98058c1-7dbd-412c-bb9c-4c61123bcdb4" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:933b1d82-9779-4e72-9de4-4940b51e3a12", + "resource": { + "resourceType": "Immunization", + "id": "933b1d82-9779-4e72-9de4-4940b51e3a12", + "status": "completed", + "date": "2012-11-01T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8c02c2a5-1e8c-4637-bd48-3c1eba65e5ef", + "resource": { + "resourceType": "Claim", + "id": "8c02c2a5-1e8c-4637-bd48-3c1eba65e5ef", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 817.84, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9", + "resource": { + "resourceType": "Encounter", + "id": "86fce1d4-25e1-4915-b644-9cfdbf32feb9", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2013-05-02T12:17:06-07:00", + "end": "2013-05-02T12:32:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b0563b7c-f103-4cde-a8bb-bacf07a87bf0", + "resource": { + "resourceType": "Observation", + "id": "b0563b7c-f103-4cde-a8bb-bacf07a87bf0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 87.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c3990c74-0493-4fc5-98ed-60eb5598b655", + "resource": { + "resourceType": "Observation", + "id": "c3990c74-0493-4fc5-98ed-60eb5598b655", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e08634b3-4e86-4ae4-9aa2-093c790d6f15", + "resource": { + "resourceType": "Observation", + "id": "e08634b3-4e86-4ae4-9aa2-093c790d6f15", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 13, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:39038ef5-60d5-446b-a2b6-b60d579c4453", + "resource": { + "resourceType": "Observation", + "id": "39038ef5-60d5-446b-a2b6-b60d579c4453", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 70.606, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7a7f1b5b-5fa2-41f5-b709-8e193b4e71a7", + "resource": { + "resourceType": "Observation", + "id": "7a7f1b5b-5fa2-41f5-b709-8e193b4e71a7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 47.760, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3ddea093-5118-46c0-bfc0-696301172acf", + "resource": { + "resourceType": "Observation", + "id": "3ddea093-5118-46c0-bfc0-696301172acf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 17.150, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5035e3b-e2a1-4e60-a1c3-825d9350230d", + "resource": { + "resourceType": "Observation", + "id": "a5035e3b-e2a1-4e60-a1c3-825d9350230d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 74.062, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e2324eb-9a70-414a-8daf-83a9a6b31566", + "resource": { + "resourceType": "Observation", + "id": "5e2324eb-9a70-414a-8daf-83a9a6b31566", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 128, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8755f73-ec5e-4c87-b6f6-388b8ad4728b", + "resource": { + "resourceType": "Observation", + "id": "e8755f73-ec5e-4c87-b6f6-388b8ad4728b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 73, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b588520-047b-44d2-be15-8a7b79d3bee2", + "resource": { + "resourceType": "Observation", + "id": "0b588520-047b-44d2-be15-8a7b79d3bee2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68a9b083-74f4-4522-a57e-6a37aa19539c", + "resource": { + "resourceType": "Observation", + "id": "68a9b083-74f4-4522-a57e-6a37aa19539c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "effectiveDateTime": "2013-05-02T12:17:06-07:00", + "issued": "2013-05-02T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:907ae086-6c1b-4028-a658-ac1ec4bb5dd6", + "resource": { + "resourceType": "MedicationOrder", + "id": "907ae086-6c1b-4028-a658-ac1ec4bb5dd6", + "dateWritten": "2013-05-02T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:56958178-94cc-48dc-b936-5183ab1d12d0", + "resource": { + "resourceType": "Claim", + "id": "56958178-94cc-48dc-b936-5183ab1d12d0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:907ae086-6c1b-4028-a658-ac1ec4bb5dd6" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3dfe1813-2a01-4590-8087-92ad9f9bb1d0", + "resource": { + "resourceType": "MedicationOrder", + "id": "3dfe1813-2a01-4590-8087-92ad9f9bb1d0", + "dateWritten": "2013-05-02T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:dd41427a-e723-414d-a53b-c9138e559daa", + "resource": { + "resourceType": "Claim", + "id": "dd41427a-e723-414d-a53b-c9138e559daa", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:3dfe1813-2a01-4590-8087-92ad9f9bb1d0" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:30cf44a8-f2bb-47c4-a946-063dc4c69e83", + "resource": { + "resourceType": "MedicationOrder", + "id": "30cf44a8-f2bb-47c4-a946-063dc4c69e83", + "dateWritten": "2013-05-02T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:e9fb7e56-7606-4838-9899-8f40dee4b729", + "resource": { + "resourceType": "Claim", + "id": "e9fb7e56-7606-4838-9899-8f40dee4b729", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:30cf44a8-f2bb-47c4-a946-063dc4c69e83" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f423073c-4b1c-4d42-a7ac-f8878e4ad24b", + "resource": { + "resourceType": "MedicationOrder", + "id": "f423073c-4b1c-4d42-a7ac-f8878e4ad24b", + "dateWritten": "2013-05-02T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:1dbfdcf9-280c-4b5d-a24a-3a0d0c8808e1", + "resource": { + "resourceType": "Claim", + "id": "1dbfdcf9-280c-4b5d-a24a-3a0d0c8808e1", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:f423073c-4b1c-4d42-a7ac-f8878e4ad24b" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:137807f4-5555-4a52-93bb-81307f85b521", + "resource": { + "resourceType": "Immunization", + "id": "137807f4-5555-4a52-93bb-81307f85b521", + "status": "completed", + "date": "2013-05-02T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4e717831-f892-44ad-8355-3a6e05eb8618", + "resource": { + "resourceType": "Claim", + "id": "4e717831-f892-44ad-8355-3a6e05eb8618", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71", + "resource": { + "resourceType": "Encounter", + "id": "29673ff6-9827-48ce-9bcd-9fa3e0e50f71", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2013-10-31T12:17:06-07:00", + "end": "2013-10-31T12:47:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bff987d7-f74c-4506-8ca8-6940d5fb05a4", + "resource": { + "resourceType": "Observation", + "id": "bff987d7-f74c-4506-8ca8-6940d5fb05a4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 91.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1359ec09-da6d-43d5-9477-13d2076b6328", + "resource": { + "resourceType": "Observation", + "id": "1359ec09-da6d-43d5-9477-13d2076b6328", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:54507301-4101-4cf8-97e2-abac21828a63", + "resource": { + "resourceType": "Observation", + "id": "54507301-4101-4cf8-97e2-abac21828a63", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 13.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3ac3ae97-437b-4f23-bebe-157dea93f661", + "resource": { + "resourceType": "Observation", + "id": "3ac3ae97-437b-4f23-bebe-157dea93f661", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 68.648, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c3956f9-77ff-49c5-8ef4-98effb797ab4", + "resource": { + "resourceType": "Observation", + "id": "2c3956f9-77ff-49c5-8ef4-98effb797ab4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 48.070, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:837c31f3-6766-4098-87ed-6e4b9d87d928", + "resource": { + "resourceType": "Observation", + "id": "837c31f3-6766-4098-87ed-6e4b9d87d928", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 16.780, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fd029216-5fc1-4500-9912-3c1a2e78825d", + "resource": { + "resourceType": "Observation", + "id": "fd029216-5fc1-4500-9912-3c1a2e78825d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 72.591, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2588f404-45f3-4789-8a3c-08500cad701e", + "resource": { + "resourceType": "Observation", + "id": "2588f404-45f3-4789-8a3c-08500cad701e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 123, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0622c5e-ea7d-453b-9a46-625e39dc7a54", + "resource": { + "resourceType": "Observation", + "id": "b0622c5e-ea7d-453b-9a46-625e39dc7a54", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 88, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d85956e-6682-4d4e-b84e-bc63e0b73b6b", + "resource": { + "resourceType": "Observation", + "id": "7d85956e-6682-4d4e-b84e-bc63e0b73b6b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff65eaaa-ed71-4863-9c32-defa9bbfa409", + "resource": { + "resourceType": "Observation", + "id": "ff65eaaa-ed71-4863-9c32-defa9bbfa409", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "effectiveDateTime": "2013-10-31T12:17:06-07:00", + "issued": "2013-10-31T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b7100ec-3ce9-4da7-b21f-80b2ed537330", + "resource": { + "resourceType": "Procedure", + "id": "6b7100ec-3ce9-4da7-b21f-80b2ed537330", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2013-10-31T12:17:06-07:00", + "end": "2013-10-31T12:32:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:63beaa22-0d81-4a73-97ea-d67ece4f01df", + "resource": { + "resourceType": "MedicationOrder", + "id": "63beaa22-0d81-4a73-97ea-d67ece4f01df", + "dateWritten": "2013-10-31T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:edd87983-8719-468c-aa41-ee42aecdd09c", + "resource": { + "resourceType": "Claim", + "id": "edd87983-8719-468c-aa41-ee42aecdd09c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:63beaa22-0d81-4a73-97ea-d67ece4f01df" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1aadd4b8-6c66-430f-83c6-7cd3c60c7f90", + "resource": { + "resourceType": "MedicationOrder", + "id": "1aadd4b8-6c66-430f-83c6-7cd3c60c7f90", + "dateWritten": "2013-10-31T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:09fe6cac-1c69-46e2-8f6c-4d02ffe0ef66", + "resource": { + "resourceType": "Claim", + "id": "09fe6cac-1c69-46e2-8f6c-4d02ffe0ef66", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:1aadd4b8-6c66-430f-83c6-7cd3c60c7f90" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:678ad620-5b9a-4352-9f23-29cf9c3a912c", + "resource": { + "resourceType": "MedicationOrder", + "id": "678ad620-5b9a-4352-9f23-29cf9c3a912c", + "dateWritten": "2013-10-31T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:b190063c-e1aa-414b-b243-64d73f79e1af", + "resource": { + "resourceType": "Claim", + "id": "b190063c-e1aa-414b-b243-64d73f79e1af", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:678ad620-5b9a-4352-9f23-29cf9c3a912c" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:17a05c27-33da-44f9-996a-4430cd0d7e31", + "resource": { + "resourceType": "MedicationOrder", + "id": "17a05c27-33da-44f9-996a-4430cd0d7e31", + "dateWritten": "2013-10-31T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:756fbfe9-c7d7-4f1d-bf5b-a2c9d4df78c4", + "resource": { + "resourceType": "Claim", + "id": "756fbfe9-c7d7-4f1d-bf5b-a2c9d4df78c4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:17a05c27-33da-44f9-996a-4430cd0d7e31" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:92917a82-cb51-4098-a4b3-ae37939a35dd", + "resource": { + "resourceType": "Immunization", + "id": "92917a82-cb51-4098-a4b3-ae37939a35dd", + "status": "completed", + "date": "2013-10-31T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:24941710-6135-4b5b-aade-c5eeffb63843", + "resource": { + "resourceType": "Claim", + "id": "24941710-6135-4b5b-aade-c5eeffb63843", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 562.93, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3", + "resource": { + "resourceType": "Encounter", + "id": "fe8a9231-bcca-4236-88ca-d37957944cd3", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2014-05-01T12:17:06-07:00", + "end": "2014-05-01T12:32:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4f50079a-1238-4d00-949d-b79b114dd934", + "resource": { + "resourceType": "Observation", + "id": "4f50079a-1238-4d00-949d-b79b114dd934", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 94.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bd68506-8b8f-48ad-a313-2bcb8a0fc968", + "resource": { + "resourceType": "Observation", + "id": "7bd68506-8b8f-48ad-a313-2bcb8a0fc968", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ce72f5ed-e116-4da6-a5b0-c5ab069c06ed", + "resource": { + "resourceType": "Observation", + "id": "ce72f5ed-e116-4da6-a5b0-c5ab069c06ed", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 14.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b076ad63-5db5-4f9b-afeb-46750f0ac4ab", + "resource": { + "resourceType": "Observation", + "id": "b076ad63-5db5-4f9b-afeb-46750f0ac4ab", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 67.192, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d7400b50-b800-4ac9-9ba4-c7a7bb0b4941", + "resource": { + "resourceType": "Observation", + "id": "d7400b50-b800-4ac9-9ba4-c7a7bb0b4941", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 48.090, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68f8d965-4ec2-4e16-b138-606ed985d882", + "resource": { + "resourceType": "Observation", + "id": "68f8d965-4ec2-4e16-b138-606ed985d882", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 16.650, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b0217b1-bc86-4579-ae82-a0313b1c0732", + "resource": { + "resourceType": "Observation", + "id": "4b0217b1-bc86-4579-ae82-a0313b1c0732", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 75.283, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a48ea91d-7651-41d7-9e56-9a1359de3c4b", + "resource": { + "resourceType": "Observation", + "id": "a48ea91d-7651-41d7-9e56-9a1359de3c4b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 112, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6da220e0-be3a-4e05-8482-dd65c26d2a9e", + "resource": { + "resourceType": "Observation", + "id": "6da220e0-be3a-4e05-8482-dd65c26d2a9e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 80, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33d7a8b6-1941-4429-8ca9-f906a362ca95", + "resource": { + "resourceType": "Observation", + "id": "33d7a8b6-1941-4429-8ca9-f906a362ca95", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92858dce-3f4e-448b-8bfa-25af37db5a16", + "resource": { + "resourceType": "Observation", + "id": "92858dce-3f4e-448b-8bfa-25af37db5a16", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "effectiveDateTime": "2014-05-01T12:17:06-07:00", + "issued": "2014-05-01T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc", + "resource": { + "resourceType": "MedicationOrder", + "id": "dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc", + "dateWritten": "2014-05-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:86e00ca5-98a6-462a-953e-da70438ed2bc", + "resource": { + "resourceType": "Claim", + "id": "86e00ca5-98a6-462a-953e-da70438ed2bc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fd888874-0be5-4e2c-a572-aa50fcde16ba", + "resource": { + "resourceType": "MedicationOrder", + "id": "fd888874-0be5-4e2c-a572-aa50fcde16ba", + "dateWritten": "2014-05-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:f52a3344-b123-417e-90bb-b2229009db7c", + "resource": { + "resourceType": "Claim", + "id": "f52a3344-b123-417e-90bb-b2229009db7c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:fd888874-0be5-4e2c-a572-aa50fcde16ba" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0521e4ee-2112-4099-8d3e-119302e8a5bf", + "resource": { + "resourceType": "MedicationOrder", + "id": "0521e4ee-2112-4099-8d3e-119302e8a5bf", + "dateWritten": "2014-05-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:a41b7f4c-78e6-4078-9523-fb49108d10bf", + "resource": { + "resourceType": "Claim", + "id": "a41b7f4c-78e6-4078-9523-fb49108d10bf", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:0521e4ee-2112-4099-8d3e-119302e8a5bf" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:842bb8ba-e02d-4935-8fe5-3573b6c2836e", + "resource": { + "resourceType": "MedicationOrder", + "id": "842bb8ba-e02d-4935-8fe5-3573b6c2836e", + "dateWritten": "2014-05-01T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:a16e708f-062b-4547-85a6-f8abb56c76fe", + "resource": { + "resourceType": "Claim", + "id": "a16e708f-062b-4547-85a6-f8abb56c76fe", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:842bb8ba-e02d-4935-8fe5-3573b6c2836e" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:10259cfd-2bf8-4a73-804a-1bf0bf5b6126", + "resource": { + "resourceType": "Claim", + "id": "10259cfd-2bf8-4a73-804a-1bf0bf5b6126", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4", + "resource": { + "resourceType": "Encounter", + "id": "6b54f746-9c2e-4a36-a384-59c755778ce4", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2014-10-30T12:17:06-07:00", + "end": "2014-10-30T12:47:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6a35034f-5616-4f06-a2d0-1c7ce2eedb9a", + "resource": { + "resourceType": "Observation", + "id": "6a35034f-5616-4f06-a2d0-1c7ce2eedb9a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 97.900, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:395b9a64-6a17-4a45-beab-955f0262aeb4", + "resource": { + "resourceType": "Observation", + "id": "395b9a64-6a17-4a45-beab-955f0262aeb4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:086a10f7-18dc-4b56-ae52-9a7871b9c258", + "resource": { + "resourceType": "Observation", + "id": "086a10f7-18dc-4b56-ae52-9a7871b9c258", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 15.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0018d3fe-276d-4f04-aad5-f17cf91331b1", + "resource": { + "resourceType": "Observation", + "id": "0018d3fe-276d-4f04-aad5-f17cf91331b1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 67.192, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83888cce-9797-4e11-9ad9-48f2bee938e3", + "resource": { + "resourceType": "Observation", + "id": "83888cce-9797-4e11-9ad9-48f2bee938e3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 48.090, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:414f6fb4-bdef-4458-9e4c-c8c7b5da92ef", + "resource": { + "resourceType": "Observation", + "id": "414f6fb4-bdef-4458-9e4c-c8c7b5da92ef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 16.610, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d64fc45b-4be9-43ba-b3ef-79ec46006e82", + "resource": { + "resourceType": "Observation", + "id": "d64fc45b-4be9-43ba-b3ef-79ec46006e82", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 78.526, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a8567ef0-11ee-45ec-b6fb-79991775c11c", + "resource": { + "resourceType": "Observation", + "id": "a8567ef0-11ee-45ec-b6fb-79991775c11c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 103, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2da9a643-2834-4d2e-9606-36327310a35f", + "resource": { + "resourceType": "Observation", + "id": "2da9a643-2834-4d2e-9606-36327310a35f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:657a7b45-dfc8-4876-8c6f-3beeceda6948", + "resource": { + "resourceType": "Observation", + "id": "657a7b45-dfc8-4876-8c6f-3beeceda6948", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2be0c9be-5fee-4d62-8534-134722a6c56d", + "resource": { + "resourceType": "Observation", + "id": "2be0c9be-5fee-4d62-8534-134722a6c56d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "effectiveDateTime": "2014-10-30T12:17:06-07:00", + "issued": "2014-10-30T12:17:06.181-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3ef4fa21-4101-4510-b0bc-8971b7b6ac3e", + "resource": { + "resourceType": "Procedure", + "id": "3ef4fa21-4101-4510-b0bc-8971b7b6ac3e", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2014-10-30T12:17:06-07:00", + "end": "2014-10-30T12:32:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:33faacab-5f2f-4c4c-8e4b-5dc2762a6b90", + "resource": { + "resourceType": "MedicationOrder", + "id": "33faacab-5f2f-4c4c-8e4b-5dc2762a6b90", + "dateWritten": "2014-10-30T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:f87bf469-3e34-4ddd-b252-dcc93cc029e4", + "resource": { + "resourceType": "Claim", + "id": "f87bf469-3e34-4ddd-b252-dcc93cc029e4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:33faacab-5f2f-4c4c-8e4b-5dc2762a6b90" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a516b9fd-5cd6-419f-a13a-340c868350b7", + "resource": { + "resourceType": "MedicationOrder", + "id": "a516b9fd-5cd6-419f-a13a-340c868350b7", + "dateWritten": "2014-10-30T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:254a8257-c61d-407d-a7b6-665408cbb5ad", + "resource": { + "resourceType": "Claim", + "id": "254a8257-c61d-407d-a7b6-665408cbb5ad", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a516b9fd-5cd6-419f-a13a-340c868350b7" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:21a17190-cf83-4102-a448-48ac25918efe", + "resource": { + "resourceType": "MedicationOrder", + "id": "21a17190-cf83-4102-a448-48ac25918efe", + "dateWritten": "2014-10-30T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:75307869-f3bb-4c65-bd91-b4379b98b116", + "resource": { + "resourceType": "Claim", + "id": "75307869-f3bb-4c65-bd91-b4379b98b116", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:21a17190-cf83-4102-a448-48ac25918efe" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7a2de518-de8b-400f-98aa-bbdd17c21aa6", + "resource": { + "resourceType": "MedicationOrder", + "id": "7a2de518-de8b-400f-98aa-bbdd17c21aa6", + "dateWritten": "2014-10-30T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:44bb4b83-b03c-44c1-b3c0-bff00599c44d", + "resource": { + "resourceType": "Claim", + "id": "44bb4b83-b03c-44c1-b3c0-bff00599c44d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:7a2de518-de8b-400f-98aa-bbdd17c21aa6" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8cbad22a-0830-4fce-afa8-b3cfbacc5e3f", + "resource": { + "resourceType": "Immunization", + "id": "8cbad22a-0830-4fce-afa8-b3cfbacc5e3f", + "status": "completed", + "date": "2014-10-30T12:17:06-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ca29355e-1b2c-4358-b8fb-8df2b1a9ecfc", + "resource": { + "resourceType": "Claim", + "id": "ca29355e-1b2c-4358-b8fb-8df2b1a9ecfc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 867.25, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383", + "resource": { + "resourceType": "Encounter", + "id": "3071a31a-0bb1-4a7c-828c-1ea626f28383", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2015-11-05T11:17:06-08:00", + "end": "2015-11-05T11:32:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f69a0e97-6390-436a-b6e4-eb3afb4e0f60", + "resource": { + "resourceType": "Observation", + "id": "f69a0e97-6390-436a-b6e4-eb3afb4e0f60", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 104.20, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8c359636-b990-4ffc-8b21-2d8f1733f6c6", + "resource": { + "resourceType": "Observation", + "id": "8c359636-b990-4ffc-8b21-2d8f1733f6c6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a53bb911-3e18-4199-a12b-63efe183f897", + "resource": { + "resourceType": "Observation", + "id": "a53bb911-3e18-4199-a12b-63efe183f897", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 18.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b757eed6-3a53-42af-a532-c4567d864337", + "resource": { + "resourceType": "Observation", + "id": "b757eed6-3a53-42af-a532-c4567d864337", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 17.370, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba466ba4-f1fa-4b4c-ab63-c9d5e5a28770", + "resource": { + "resourceType": "Observation", + "id": "ba466ba4-f1fa-4b4c-ab63-c9d5e5a28770", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 91.180, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:055230ca-681a-466a-a0ed-b208b6ba592b", + "resource": { + "resourceType": "Observation", + "id": "055230ca-681a-466a-a0ed-b208b6ba592b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:044bc207-c84d-4692-b1c2-672701561f5a", + "resource": { + "resourceType": "Observation", + "id": "044bc207-c84d-4692-b1c2-672701561f5a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 66, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf2b2c65-a10d-44dd-9159-80a85e92b111", + "resource": { + "resourceType": "Observation", + "id": "cf2b2c65-a10d-44dd-9159-80a85e92b111", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f1f8911-f9f5-4f10-9ac7-0dc5a7a618b4", + "resource": { + "resourceType": "Observation", + "id": "5f1f8911-f9f5-4f10-9ac7-0dc5a7a618b4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "effectiveDateTime": "2015-11-05T11:17:06-08:00", + "issued": "2015-11-05T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a28093d4-165c-47ba-b68e-60305247601f", + "resource": { + "resourceType": "MedicationOrder", + "id": "a28093d4-165c-47ba-b68e-60305247601f", + "dateWritten": "2015-11-05T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:df327e18-edbc-408c-abc2-2107c9850382", + "resource": { + "resourceType": "Claim", + "id": "df327e18-edbc-408c-abc2-2107c9850382", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a28093d4-165c-47ba-b68e-60305247601f" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3339c556-439b-4e53-9be2-e16a89d6549a", + "resource": { + "resourceType": "MedicationOrder", + "id": "3339c556-439b-4e53-9be2-e16a89d6549a", + "dateWritten": "2015-11-05T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:eb2ac11d-eea3-4469-9154-35facdafa11e", + "resource": { + "resourceType": "Claim", + "id": "eb2ac11d-eea3-4469-9154-35facdafa11e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:3339c556-439b-4e53-9be2-e16a89d6549a" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:915d3f1e-4b07-4733-af36-a761af333ea3", + "resource": { + "resourceType": "MedicationOrder", + "id": "915d3f1e-4b07-4733-af36-a761af333ea3", + "dateWritten": "2015-11-05T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:5d11b35b-8f90-4b98-a36f-d943c965673a", + "resource": { + "resourceType": "Claim", + "id": "5d11b35b-8f90-4b98-a36f-d943c965673a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:915d3f1e-4b07-4733-af36-a761af333ea3" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6a8cd67a-a9df-4d00-9f05-73facb256ff1", + "resource": { + "resourceType": "MedicationOrder", + "id": "6a8cd67a-a9df-4d00-9f05-73facb256ff1", + "dateWritten": "2015-11-05T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:4aab4355-1444-4e35-982f-3fa704301fff", + "resource": { + "resourceType": "Claim", + "id": "4aab4355-1444-4e35-982f-3fa704301fff", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:6a8cd67a-a9df-4d00-9f05-73facb256ff1" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:051bf2d4-3dea-49aa-b51f-5abfc2bc45dc", + "resource": { + "resourceType": "Immunization", + "id": "051bf2d4-3dea-49aa-b51f-5abfc2bc45dc", + "status": "completed", + "date": "2015-11-05T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d8408600-dfb8-46dd-b7ab-1cd897e4cb3a", + "resource": { + "resourceType": "Immunization", + "id": "d8408600-dfb8-46dd-b7ab-1cd897e4cb3a", + "status": "completed", + "date": "2015-11-05T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d2ea0bad-affb-41ef-bbf4-9563fc1ffde7", + "resource": { + "resourceType": "Immunization", + "id": "d2ea0bad-affb-41ef-bbf4-9563fc1ffde7", + "status": "completed", + "date": "2015-11-05T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:808fd435-e586-47e6-8e87-2efa63e6de19", + "resource": { + "resourceType": "Immunization", + "id": "808fd435-e586-47e6-8e87-2efa63e6de19", + "status": "completed", + "date": "2015-11-05T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:88696441-9d6a-432f-8504-d62a81e2548d", + "resource": { + "resourceType": "Immunization", + "id": "88696441-9d6a-432f-8504-d62a81e2548d", + "status": "completed", + "date": "2015-11-05T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:bcc66b34-3531-41da-8ce4-2ec28a3d21a6", + "resource": { + "resourceType": "Claim", + "id": "bcc66b34-3531-41da-8ce4-2ec28a3d21a6", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3", + "resource": { + "resourceType": "Encounter", + "id": "533c729b-7180-41ef-b222-65a1ad95bad3", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2016-11-10T11:17:06-08:00", + "end": "2016-11-10T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d8adac1d-6f28-4544-9a93-abea9e4f1255", + "resource": { + "resourceType": "Observation", + "id": "d8adac1d-6f28-4544-9a93-abea9e4f1255", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 110.30, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:240db449-ebc7-43f0-b667-43fe68c113b8", + "resource": { + "resourceType": "Observation", + "id": "240db449-ebc7-43f0-b667-43fe68c113b8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cdc17c05-281c-4d65-8b77-f6bf318ccb87", + "resource": { + "resourceType": "Observation", + "id": "cdc17c05-281c-4d65-8b77-f6bf318ccb87", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 22.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2c2837d-34b8-4de8-bc37-66c04123daee", + "resource": { + "resourceType": "Observation", + "id": "e2c2837d-34b8-4de8-bc37-66c04123daee", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 18.740, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:22485690-c64c-4a15-aea8-df046fdcb8cc", + "resource": { + "resourceType": "Observation", + "id": "22485690-c64c-4a15-aea8-df046fdcb8cc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 96.257, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5781b1ab-02a4-416c-993b-3c8da371c4d5", + "resource": { + "resourceType": "Observation", + "id": "5781b1ab-02a4-416c-993b-3c8da371c4d5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c9ae8ca-9b83-4bda-8c58-4bdd43ab18f5", + "resource": { + "resourceType": "Observation", + "id": "9c9ae8ca-9b83-4bda-8c58-4bdd43ab18f5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 87, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25bf47f2-324c-456c-9fce-d40825d9e4db", + "resource": { + "resourceType": "Observation", + "id": "25bf47f2-324c-456c-9fce-d40825d9e4db", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecdfba49-8905-49bf-b774-cc79814b2e96", + "resource": { + "resourceType": "Observation", + "id": "ecdfba49-8905-49bf-b774-cc79814b2e96", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 7.2494, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:575bb625-0686-4cb0-85b2-979d7e4dd3d7", + "resource": { + "resourceType": "Observation", + "id": "575bb625-0686-4cb0-85b2-979d7e4dd3d7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 4.1674, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c248f3a7-3cb3-479b-9229-18caf8495d3d", + "resource": { + "resourceType": "Observation", + "id": "c248f3a7-3cb3-479b-9229-18caf8495d3d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 14.995, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1ce1f94-f907-4605-9c7b-647168c8242b", + "resource": { + "resourceType": "Observation", + "id": "a1ce1f94-f907-4605-9c7b-647168c8242b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 39.933, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:243bf02c-d6f5-4c3c-b5fc-ccba669f6b8e", + "resource": { + "resourceType": "Observation", + "id": "243bf02c-d6f5-4c3c-b5fc-ccba669f6b8e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 80.178, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3cdad9b2-23d2-4fad-93b1-0db950b4b4a0", + "resource": { + "resourceType": "Observation", + "id": "3cdad9b2-23d2-4fad-93b1-0db950b4b4a0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 28.351, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f24d4e5-457f-47de-a4ff-e29dedce4263", + "resource": { + "resourceType": "Observation", + "id": "1f24d4e5-457f-47de-a4ff-e29dedce4263", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 35.565, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa3337bc-9376-4bec-ae1a-6b4f43fd1251", + "resource": { + "resourceType": "Observation", + "id": "fa3337bc-9376-4bec-ae1a-6b4f43fd1251", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 41.654, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6415dfc-9e36-4bb2-87fa-b6b09ec777db", + "resource": { + "resourceType": "Observation", + "id": "e6415dfc-9e36-4bb2-87fa-b6b09ec777db", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 222.68, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d6c05e8-dc0b-4574-b2b6-a5ad00c765e4", + "resource": { + "resourceType": "Observation", + "id": "3d6c05e8-dc0b-4574-b2b6-a5ad00c765e4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 508.47, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:21467e46-8d7d-4191-abcd-75e9fc00fc5a", + "resource": { + "resourceType": "Observation", + "id": "21467e46-8d7d-4191-abcd-75e9fc00fc5a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueQuantity": { + "value": 10.950, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9fe386f9-da6d-4b6a-88c4-cd5fef9ea747", + "resource": { + "resourceType": "Observation", + "id": "9fe386f9-da6d-4b6a-88c4-cd5fef9ea747", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf6d858e-7dd5-4926-8712-bfcd2b27a866", + "resource": { + "resourceType": "Procedure", + "id": "cf6d858e-7dd5-4926-8712-bfcd2b27a866", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2016-11-10T11:17:06-08:00", + "end": "2016-11-10T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c3c48c3f-5e86-4a39-880a-76c153d49af7", + "resource": { + "resourceType": "MedicationOrder", + "id": "c3c48c3f-5e86-4a39-880a-76c153d49af7", + "dateWritten": "2016-11-10T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:ef45b077-97fe-4935-9021-73fd724537a9", + "resource": { + "resourceType": "Claim", + "id": "ef45b077-97fe-4935-9021-73fd724537a9", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:c3c48c3f-5e86-4a39-880a-76c153d49af7" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f33b9e8d-768c-45a1-b990-163ceb96a32b", + "resource": { + "resourceType": "MedicationOrder", + "id": "f33b9e8d-768c-45a1-b990-163ceb96a32b", + "dateWritten": "2016-11-10T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:9be6af68-849b-47dc-9f1f-6249f16a1e86", + "resource": { + "resourceType": "Claim", + "id": "9be6af68-849b-47dc-9f1f-6249f16a1e86", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:f33b9e8d-768c-45a1-b990-163ceb96a32b" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:80aa64ec-931c-4955-b1fd-d9569f059e60", + "resource": { + "resourceType": "MedicationOrder", + "id": "80aa64ec-931c-4955-b1fd-d9569f059e60", + "dateWritten": "2016-11-10T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:f6ae23d6-f5c9-482e-b814-73134aadf7e7", + "resource": { + "resourceType": "Claim", + "id": "f6ae23d6-f5c9-482e-b814-73134aadf7e7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:80aa64ec-931c-4955-b1fd-d9569f059e60" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:27a15798-b2bd-4827-9e9e-49cf65e14d60", + "resource": { + "resourceType": "MedicationOrder", + "id": "27a15798-b2bd-4827-9e9e-49cf65e14d60", + "dateWritten": "2016-11-10T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:eb5af8f9-9901-43e5-8aec-ce04ffebc6db", + "resource": { + "resourceType": "Claim", + "id": "eb5af8f9-9901-43e5-8aec-ce04ffebc6db", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:27a15798-b2bd-4827-9e9e-49cf65e14d60" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:91cd35b7-c51a-4fe3-a5c5-7190e09f2db1", + "resource": { + "resourceType": "Immunization", + "id": "91cd35b7-c51a-4fe3-a5c5-7190e09f2db1", + "status": "completed", + "date": "2016-11-10T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8cca8ef6-54a3-44e6-9502-aa55f640e173", + "resource": { + "resourceType": "DiagnosticReport", + "id": "8cca8ef6-54a3-44e6-9502-aa55f640e173", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" + }, + "effectiveDateTime": "2016-11-10T11:17:06-08:00", + "issued": "2016-11-10T11:17:06.181-08:00", + "performer": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "result": [ + { + "reference": "urn:uuid:21467e46-8d7d-4191-abcd-75e9fc00fc5a", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:fe2db6c9-0f90-4a67-8bca-d1e856aecbfd", + "resource": { + "resourceType": "Claim", + "id": "fe2db6c9-0f90-4a67-8bca-d1e856aecbfd", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 557.19, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4", + "resource": { + "resourceType": "Encounter", + "id": "6c2fb27d-edac-4580-a465-21ec775ac4b4", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2017-11-16T11:17:06-08:00", + "end": "2017-11-16T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5a110c03-b655-4048-8b41-30410dd30b2e", + "resource": { + "resourceType": "Observation", + "id": "5a110c03-b655-4048-8b41-30410dd30b2e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 116.40, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e62a2a09-f6f2-4c74-8ae0-1e7c6b907b20", + "resource": { + "resourceType": "Observation", + "id": "e62a2a09-f6f2-4c74-8ae0-1e7c6b907b20", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28163514-80d7-4a03-953a-24ecd6d0c7a1", + "resource": { + "resourceType": "Observation", + "id": "28163514-80d7-4a03-953a-24ecd6d0c7a1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 24.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:199be083-0820-4530-98e7-30520b484a49", + "resource": { + "resourceType": "Observation", + "id": "199be083-0820-4530-98e7-30520b484a49", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 18.400, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d96a45d-31f9-4052-9211-02897e01f173", + "resource": { + "resourceType": "Observation", + "id": "2d96a45d-31f9-4052-9211-02897e01f173", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 92.406, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b118da3-b917-4a4c-a4b1-ed7fcc5e0e86", + "resource": { + "resourceType": "Observation", + "id": "3b118da3-b917-4a4c-a4b1-ed7fcc5e0e86", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65e6e6f0-381c-4a77-a112-a4b8316bbce7", + "resource": { + "resourceType": "Observation", + "id": "65e6e6f0-381c-4a77-a112-a4b8316bbce7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 71, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:742e140e-6652-4dc6-bddc-59d2b9344500", + "resource": { + "resourceType": "Observation", + "id": "742e140e-6652-4dc6-bddc-59d2b9344500", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8cb5fce5-b3c5-461c-8d1b-2b30f17ffc72", + "resource": { + "resourceType": "Observation", + "id": "8cb5fce5-b3c5-461c-8d1b-2b30f17ffc72", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "effectiveDateTime": "2017-11-16T11:17:06-08:00", + "issued": "2017-11-16T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4222c16-78e9-409e-a511-17015a5312e9", + "resource": { + "resourceType": "Procedure", + "id": "c4222c16-78e9-409e-a511-17015a5312e9", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2017-11-16T11:17:06-08:00", + "end": "2017-11-16T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e34d43e2-4293-4c08-9352-ecacc35e4dc0", + "resource": { + "resourceType": "MedicationOrder", + "id": "e34d43e2-4293-4c08-9352-ecacc35e4dc0", + "dateWritten": "2017-11-16T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:f60394ca-d448-41ed-b802-6bf44e2c0b96", + "resource": { + "resourceType": "Claim", + "id": "f60394ca-d448-41ed-b802-6bf44e2c0b96", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:e34d43e2-4293-4c08-9352-ecacc35e4dc0" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4354e8e4-90bd-40aa-9b19-a2ff2f67be6b", + "resource": { + "resourceType": "MedicationOrder", + "id": "4354e8e4-90bd-40aa-9b19-a2ff2f67be6b", + "dateWritten": "2017-11-16T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:2ddaf801-4bcd-4c24-9ba9-5debc0b7ad8a", + "resource": { + "resourceType": "Claim", + "id": "2ddaf801-4bcd-4c24-9ba9-5debc0b7ad8a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:4354e8e4-90bd-40aa-9b19-a2ff2f67be6b" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dd249c62-56ec-4bec-8a33-7fbeea9bab14", + "resource": { + "resourceType": "MedicationOrder", + "id": "dd249c62-56ec-4bec-8a33-7fbeea9bab14", + "dateWritten": "2017-11-16T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:830198ff-b1d2-46e4-a6d7-de73909818c8", + "resource": { + "resourceType": "Claim", + "id": "830198ff-b1d2-46e4-a6d7-de73909818c8", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:dd249c62-56ec-4bec-8a33-7fbeea9bab14" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:79641136-2ad7-4d00-9de7-32294cd74eda", + "resource": { + "resourceType": "MedicationOrder", + "id": "79641136-2ad7-4d00-9de7-32294cd74eda", + "dateWritten": "2017-11-16T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:aa6ba46f-c48c-4eec-9fdf-07159f42f421", + "resource": { + "resourceType": "Claim", + "id": "aa6ba46f-c48c-4eec-9fdf-07159f42f421", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:79641136-2ad7-4d00-9de7-32294cd74eda" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:35b046a9-4986-4781-b7fd-88abb7a2c9cd", + "resource": { + "resourceType": "Immunization", + "id": "35b046a9-4986-4781-b7fd-88abb7a2c9cd", + "status": "completed", + "date": "2017-11-16T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:57538e1f-c700-420b-bcdd-b60f4feaf110", + "resource": { + "resourceType": "Claim", + "id": "57538e1f-c700-420b-bcdd-b60f4feaf110", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 545.88, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5", + "resource": { + "resourceType": "Encounter", + "id": "2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2018-11-22T11:17:06-08:00", + "end": "2018-11-22T11:32:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4f7d9ad8-9cb2-4330-b5be-6227cafed4f9", + "resource": { + "resourceType": "Observation", + "id": "4f7d9ad8-9cb2-4330-b5be-6227cafed4f9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 122.20, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:398a2817-acf4-4e06-8a9b-80b8cb14fd4c", + "resource": { + "resourceType": "Observation", + "id": "398a2817-acf4-4e06-8a9b-80b8cb14fd4c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9e0cca73-0ead-4cb9-8082-75cd10348096", + "resource": { + "resourceType": "Observation", + "id": "9e0cca73-0ead-4cb9-8082-75cd10348096", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 29.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0b32780-90b6-4a96-b1d4-4c0e4df31079", + "resource": { + "resourceType": "Observation", + "id": "c0b32780-90b6-4a96-b1d4-4c0e4df31079", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 19.700, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb50cbed-1da2-418d-82b8-103c5879b300", + "resource": { + "resourceType": "Observation", + "id": "fb50cbed-1da2-418d-82b8-103c5879b300", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 94.268, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f63355a7-7e03-47c2-a51b-0716be99660f", + "resource": { + "resourceType": "Observation", + "id": "f63355a7-7e03-47c2-a51b-0716be99660f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 139, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3fe321ad-2e98-4ec4-8f36-6b0329332191", + "resource": { + "resourceType": "Observation", + "id": "3fe321ad-2e98-4ec4-8f36-6b0329332191", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 66, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d3a4e85-0439-4a99-87e9-9c09836a1849", + "resource": { + "resourceType": "Observation", + "id": "7d3a4e85-0439-4a99-87e9-9c09836a1849", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b034145c-ed4d-47fc-a767-a233573a7279", + "resource": { + "resourceType": "Observation", + "id": "b034145c-ed4d-47fc-a767-a233573a7279", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "effectiveDateTime": "2018-11-22T11:17:06-08:00", + "issued": "2018-11-22T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e37c807-27e4-4dba-871f-5ac81870dff7", + "resource": { + "resourceType": "MedicationOrder", + "id": "7e37c807-27e4-4dba-871f-5ac81870dff7", + "dateWritten": "2018-11-22T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:21752dfc-cc7f-4d9b-8c88-3641b3e02310", + "resource": { + "resourceType": "Claim", + "id": "21752dfc-cc7f-4d9b-8c88-3641b3e02310", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:7e37c807-27e4-4dba-871f-5ac81870dff7" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a8c49bde-9362-4d88-9861-082583d639de", + "resource": { + "resourceType": "MedicationOrder", + "id": "a8c49bde-9362-4d88-9861-082583d639de", + "dateWritten": "2018-11-22T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:44fdb5f3-28f9-4cd2-9ac9-9ae9ada442f3", + "resource": { + "resourceType": "Claim", + "id": "44fdb5f3-28f9-4cd2-9ac9-9ae9ada442f3", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:a8c49bde-9362-4d88-9861-082583d639de" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:426e17e4-7eea-4cf5-9a6c-b37ef790b4e4", + "resource": { + "resourceType": "MedicationOrder", + "id": "426e17e4-7eea-4cf5-9a6c-b37ef790b4e4", + "dateWritten": "2018-11-22T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:defc2cb2-3ba0-40be-95ac-d78d7aa9faaa", + "resource": { + "resourceType": "Claim", + "id": "defc2cb2-3ba0-40be-95ac-d78d7aa9faaa", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:426e17e4-7eea-4cf5-9a6c-b37ef790b4e4" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b4e92f90-4457-4696-a774-ff4d38c07ca4", + "resource": { + "resourceType": "MedicationOrder", + "id": "b4e92f90-4457-4696-a774-ff4d38c07ca4", + "dateWritten": "2018-11-22T11:17:06-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:659506dd-ee0f-4708-9336-da59f570ad4a", + "resource": { + "resourceType": "Claim", + "id": "659506dd-ee0f-4708-9336-da59f570ad4a", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:b4e92f90-4457-4696-a774-ff4d38c07ca4" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:36fd48c5-a86f-462f-a797-ff90790076d6", + "resource": { + "resourceType": "Immunization", + "id": "36fd48c5-a86f-462f-a797-ff90790076d6", + "status": "completed", + "date": "2018-11-22T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2ec6ecb9-1817-45c5-9cae-9d2fbad392c2", + "resource": { + "resourceType": "Claim", + "id": "2ec6ecb9-1817-45c5-9cae-9d2fbad392c2", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c", + "resource": { + "resourceType": "Organization", + "id": "b7175ab4-bde5-3848-891b-579bccb77c7c", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "b7175ab4-bde5-3848-891b-579bccb77c7c" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "TUFTS MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "6176365000" + } + ], + "address": [ + { + "line": [ + "800 WASHINGTON STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02111", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044", + "resource": { + "resourceType": "Practitioner", + "id": "002862dc-5ff1-380e-82ad-a3cf9c436044", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "470" + } + ], + "active": true, + "name": { + "family": [ + "Orn563" + ], + "given": [ + "Serina556" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "800 WASHINGTON STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02111", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a", + "resource": { + "resourceType": "Encounter", + "id": "2d0f382b-c1c9-477e-ad42-0cdae1d1db4a", + "status": "finished", + "class": "emergency", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044" + } + } + ], + "period": { + "start": "2019-08-10T12:17:06-07:00", + "end": "2019-08-10T14:47:06-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b", + "resource": { + "resourceType": "Condition", + "id": "d8c9c5ad-1dfc-4999-bfe1-77c931570f2b", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" + }, + "dateRecorded": "2019-08-10", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58150001", + "display": "Fracture of clavicle" + } + ], + "text": "Fracture of clavicle" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2019-08-10T12:17:06-07:00", + "abatementDateTime": "2019-10-09T12:17:06-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:6d659beb-5b35-472f-97bb-e1ed3f0bdd8c", + "resource": { + "resourceType": "Procedure", + "id": "6d659beb-5b35-472f-97bb-e1ed3f0bdd8c", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "168594001", + "display": "Clavicle X-ray" + } + ], + "text": "Clavicle X-ray" + }, + "performedPeriod": { + "start": "2019-08-10T12:17:06-07:00", + "end": "2019-08-10T12:47:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:aee2aef3-09ef-4bfc-b352-0d7123e2af30", + "resource": { + "resourceType": "Procedure", + "id": "aee2aef3-09ef-4bfc-b352-0d7123e2af30", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "305428000", + "display": "Admission to orthopedic department" + } + ], + "text": "Admission to orthopedic department" + }, + "reasonReference": { + "reference": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b" + }, + "performedPeriod": { + "start": "2019-08-10T12:17:06-07:00", + "end": "2019-08-10T13:17:06-07:00" + }, + "encounter": { + "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:92365c53-0cdd-4cc5-b35b-bebb279de3ee", + "resource": { + "resourceType": "MedicationOrder", + "id": "92365c53-0cdd-4cc5-b35b-bebb279de3ee", + "dateWritten": "2019-08-10T12:17:06-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044" + }, + "encounter": { + "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:8fd60b70-ee51-4c3c-ac2b-d93c419e8e18", + "resource": { + "resourceType": "Claim", + "id": "8fd60b70-ee51-4c3c-ac2b-d93c419e8e18", + "type": "institutional", + "organization": { + "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:92365c53-0cdd-4cc5-b35b-bebb279de3ee" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9a4863d1-15d6-4779-964a-5432113a3773", + "resource": { + "resourceType": "CarePlan", + "id": "9a4863d1-15d6-4779-964a-5432113a3773", + "text": { + "status": "generated", + "div": "
    Fracture care
    " + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "context": { + "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" + }, + "period": { + "start": "2019-08-10T12:17:06-07:00", + "end": "2019-10-09T12:17:06-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385691007", + "display": "Fracture care" + } + ], + "text": "Fracture care" + } + ], + "addresses": [ + { + "reference": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "183051005", + "display": "Recommendation to rest" + } + ], + "text": "Recommendation to rest" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "408580007", + "display": "Physical activity target light exercise" + } + ], + "text": "Physical activity target light exercise" + }, + "status": "completed", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:c8157528-c44b-4964-8b9b-8e5d8d546b87", + "resource": { + "resourceType": "ImagingStudy", + "id": "c8157528-c44b-4964-8b9b-8e5d8d546b87", + "started": "2019-08-10T12:17:06-07:00", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "uid": "urn:oid:1.2.840.99999999.45493348.1586309773119", + "numberOfSeries": 1, + "numberOfInstances": 1, + "series": [ + { + "number": 1, + "modality": { + "system": "http://dicom.nema.org/resources/ontology/DCM", + "code": "DX", + "display": "Digital Radiography" + }, + "uid": "urn:oid:1.2.840.99999999.1.49310789.1586309773119", + "numberOfInstances": 1, + "availability": "UNAVAILABLE", + "bodySite": { + "system": "http://snomed.info/sct", + "code": "51299004", + "display": "Clavicle" + }, + "started": "2019-08-10T12:17:06-07:00", + "instance": [ + { + "number": 1, + "uid": "urn:oid:1.2.840.99999999.1.1.61194548.1586309773119", + "sopClass": "urn:oid:1.2.840.10008.5.1.4.1.1.1.1", + "title": "Image of clavicle" + } + ] + } + ] + }, + "request": { + "method": "POST", + "url": "ImagingStudy" + } + }, + { + "fullUrl": "urn:uuid:328f5fab-d3ff-4aa9-8be6-ea12fefd5095", + "resource": { + "resourceType": "Claim", + "id": "328f5fab-d3ff-4aa9-8be6-ea12fefd5095", + "type": "institutional", + "organization": { + "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "58150001", + "display": "Fracture of clavicle" + } + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "168594001", + "display": "Clavicle X-ray" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "305428000", + "display": "Admission to orthopedic department" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3", + "resource": { + "resourceType": "Organization", + "id": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "BOSTON MEDICAL CENTER CORPORATION-", + "telecom": [ + { + "system": "phone", + "value": "6176388000" + } + ], + "address": [ + { + "line": [ + "1 BOSTON MEDICAL CENTER PLACE" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02118", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", + "resource": { + "resourceType": "Practitioner", + "id": "64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "140" + } + ], + "active": true, + "name": { + "family": [ + "Kuvalis369" + ], + "given": [ + "Maricruz991" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "1 BOSTON MEDICAL CENTER PLACE" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02118", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:087434d3-9ce1-479e-a0f3-c5f6db5ba171", + "resource": { + "resourceType": "Encounter", + "id": "087434d3-9ce1-479e-a0f3-c5f6db5ba171", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd" + } + } + ], + "period": { + "start": "2019-10-09T12:17:06-07:00", + "end": "2019-10-09T12:32:06-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58150001", + "display": "Fracture of clavicle" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:02e30739-43b7-4a95-b94a-3e84910367fa", + "resource": { + "resourceType": "Claim", + "id": "02e30739-43b7-4a95-b94a-3e84910367fa", + "type": "institutional", + "organization": { + "reference": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce", + "resource": { + "resourceType": "Encounter", + "id": "671ab4ad-1066-4bde-b53e-492724b2f7ce", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + } + } + ], + "period": { + "start": "2019-11-28T11:17:06-08:00", + "end": "2019-11-28T11:47:06-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:953671bc-f307-46b4-8245-5320edf44237", + "resource": { + "resourceType": "Observation", + "id": "953671bc-f307-46b4-8245-5320edf44237", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 127.90, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c3f1068-32e6-41ab-9c2e-2beea381d130", + "resource": { + "resourceType": "Observation", + "id": "9c3f1068-32e6-41ab-9c2e-2beea381d130", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7a1fce1a-41a2-4eda-b38f-7fb2ad67fed9", + "resource": { + "resourceType": "Observation", + "id": "7a1fce1a-41a2-4eda-b38f-7fb2ad67fed9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 34.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:385d3583-6f5d-4456-9fd4-403ad976d46d", + "resource": { + "resourceType": "Observation", + "id": "385d3583-6f5d-4456-9fd4-403ad976d46d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 20.980, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2eb36260-32fa-42cf-a0d9-1bb621f6fd77", + "resource": { + "resourceType": "Observation", + "id": "2eb36260-32fa-42cf-a0d9-1bb621f6fd77", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 94.779, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fec7f1f1-0376-4397-aeb8-259de0c0bec3", + "resource": { + "resourceType": "Observation", + "id": "fec7f1f1-0376-4397-aeb8-259de0c0bec3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 109, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:adbe6725-fd67-499e-b7ff-c5d74b17e658", + "resource": { + "resourceType": "Observation", + "id": "adbe6725-fd67-499e-b7ff-c5d74b17e658", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 60, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2034e61f-7ee0-4543-9616-45e80ae7a7c9", + "resource": { + "resourceType": "Observation", + "id": "2034e61f-7ee0-4543-9616-45e80ae7a7c9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0e4c867-026d-4147-ab03-915d7bb49730", + "resource": { + "resourceType": "Observation", + "id": "b0e4c867-026d-4147-ab03-915d7bb49730", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "effectiveDateTime": "2019-11-28T11:17:06-08:00", + "issued": "2019-11-28T11:17:06.181-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f48d78f9-fa02-45e5-ba14-8a608a2f4d73", + "resource": { + "resourceType": "Procedure", + "id": "f48d78f9-fa02-45e5-ba14-8a608a2f4d73", + "subject": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2019-11-28T11:17:06-08:00", + "end": "2019-11-28T11:32:06-08:00" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5871c437-3957-4f84-aa19-0818bd4d52ae", + "resource": { + "resourceType": "MedicationOrder", + "id": "5871c437-3957-4f84-aa19-0818bd4d52ae", + "dateWritten": "2019-11-28T11:17:06-08:00", + "status": "active", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309362", + "display": "Clopidogrel 75 MG Oral Tablet" + } + ], + "text": "Clopidogrel 75 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:3d2e5927-6707-442a-a1cc-fe42c7ba7c81", + "resource": { + "resourceType": "Claim", + "id": "3d2e5927-6707-442a-a1cc-fe42c7ba7c81", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:5871c437-3957-4f84-aa19-0818bd4d52ae" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:49f918c7-1033-44da-9599-993bef505b71", + "resource": { + "resourceType": "MedicationOrder", + "id": "49f918c7-1033-44da-9599-993bef505b71", + "dateWritten": "2019-11-28T11:17:06-08:00", + "status": "active", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "705129", + "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + ], + "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:8467dda5-39a4-4b18-a907-5bfdf3891b80", + "resource": { + "resourceType": "Claim", + "id": "8467dda5-39a4-4b18-a907-5bfdf3891b80", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:49f918c7-1033-44da-9599-993bef505b71" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2fbd2f03-35d4-48c9-8313-be8bebbc2e5a", + "resource": { + "resourceType": "MedicationOrder", + "id": "2fbd2f03-35d4-48c9-8313-be8bebbc2e5a", + "dateWritten": "2019-11-28T11:17:06-08:00", + "status": "active", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312961", + "display": "Simvastatin 20 MG Oral Tablet" + } + ], + "text": "Simvastatin 20 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:97313763-78ef-43b2-9055-5d3eddda39e3", + "resource": { + "resourceType": "Claim", + "id": "97313763-78ef-43b2-9055-5d3eddda39e3", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:2fbd2f03-35d4-48c9-8313-be8bebbc2e5a" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4ccb9509-e4c5-4514-b36d-d239b93fdace", + "resource": { + "resourceType": "MedicationOrder", + "id": "4ccb9509-e4c5-4514-b36d-d239b93fdace", + "dateWritten": "2019-11-28T11:17:06-08:00", + "status": "active", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "prescriber": { + "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" + }, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "197361", + "display": "Amlodipine 5 MG Oral Tablet" + } + ], + "text": "Amlodipine 5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:ffdd6a17-fab2-4402-a7f9-7e306ef1e9b3", + "resource": { + "resourceType": "Claim", + "id": "ffdd6a17-fab2-4402-a7f9-7e306ef1e9b3", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:4ccb9509-e4c5-4514-b36d-d239b93fdace" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9dba0685-9a01-4bd1-b3a2-6f75960af8c1", + "resource": { + "resourceType": "Immunization", + "id": "9dba0685-9a01-4bd1-b3a2-6f75960af8c1", + "status": "completed", + "date": "2019-11-28T11:17:06-08:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d8a265a5-6188-4f85-8828-7fb12be9995b", + "resource": { + "resourceType": "Claim", + "id": "d8a265a5-6188-4f85-8828-7fb12be9995b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 473.18, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json new file mode 100644 index 000000000000..67e63f747028 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json @@ -0,0 +1,9279 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e", + "resource": { + "resourceType": "Patient", + "id": "8ed31d3e-7352-4055-89c4-f017db3f594e", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -1966016355271311648 Population seed: 1586309754086
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Race", + "code": "2106-3", + "display": "White" + } + ], + "text": "White" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Ethnicity", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + ], + "text": "Not Hispanic or Latino" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Reda120 Franecki195" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Wrentham", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.9331359488509279 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 57.066864051149075 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR" + } + ] + }, + "system": "http://hospital.smarthealthit.org", + "value": "8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB" + } + ] + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-87-6568" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "DL" + } + ] + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99945489" + } + ], + "name": [ + { + "use": "official", + "family": [ + "Bosco882" + ], + "given": [ + "Kortney212" + ], + "prefix": [ + "Ms." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-409-5117", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1961-06-28", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.422646574026295 + }, + { + "url": "longitude", + "valueDecimal": -71.18816183585602 + } + ] + } + ], + "line": [ + "475 Parker Rapid" + ], + "city": "Winchester", + "state": "Massachusetts", + "postalCode": "01890", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S" + } + ] + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219", + "resource": { + "resourceType": "Organization", + "id": "245f252c-be87-3017-8a0b-a04448a97219", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "245f252c-be87-3017-8a0b-a04448a97219" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "PCP45531", + "telecom": [ + { + "system": "phone", + "value": "617-230-9940" + } + ], + "address": [ + { + "line": [ + "661 MASSACHUSETTS AVE" + ], + "city": "ARLINGTON", + "state": "MA", + "postalCode": "02476-5001", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240", + "resource": { + "resourceType": "Practitioner", + "id": "8e9a1427-af53-3468-9d67-db3b9191c240", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "28020" + } + ], + "active": true, + "name": { + "family": [ + "Waters156" + ], + "given": [ + "Damien170" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "661 MASSACHUSETTS AVE" + ], + "city": "ARLINGTON", + "state": "MA", + "postalCode": "02476-5001", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:4512de23-4f75-4dd9-8ed0-48dd04d7e883", + "resource": { + "resourceType": "Encounter", + "id": "4512de23-4f75-4dd9-8ed0-48dd04d7e883", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "1989-09-06T12:00:58-07:00", + "end": "1989-09-06T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ea898089-97dc-4f22-a259-8d8d1d4dd5a1", + "resource": { + "resourceType": "Condition", + "id": "ea898089-97dc-4f22-a259-8d8d1d4dd5a1", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4512de23-4f75-4dd9-8ed0-48dd04d7e883" + }, + "dateRecorded": "1989-09-06", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "1989-09-06T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:41d47904-32ae-4b13-8ac9-ec8ca0bfb521", + "resource": { + "resourceType": "Claim", + "id": "41d47904-32ae-4b13-8ac9-ec8ca0bfb521", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7", + "resource": { + "resourceType": "Organization", + "id": "f4e7709c-02f6-37ca-aeea-8247d74e88e7", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "WINCHESTER HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "7817299000" + } + ], + "address": [ + { + "line": [ + "41 HIGHLAND AVENUE" + ], + "city": "WINCHESTER", + "state": "MA", + "postalCode": "01890", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203", + "resource": { + "resourceType": "Practitioner", + "id": "a0153fac-7137-30d8-bb5a-0cb7af968203", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "430" + } + ], + "active": true, + "name": { + "family": [ + "Carroll471" + ], + "given": [ + "Emmitt44" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "41 HIGHLAND AVENUE" + ], + "city": "WINCHESTER", + "state": "MA", + "postalCode": "01890", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290", + "resource": { + "resourceType": "Encounter", + "id": "99e80e98-cc07-431a-855e-983e509b9290", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2010-05-19T12:00:58-07:00", + "end": "2010-05-19T12:45:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee", + "resource": { + "resourceType": "Condition", + "id": "d1813c80-97de-4641-b3d9-e0ddede862ee", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" + }, + "dateRecorded": "2010-05-19", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ], + "text": "Normal pregnancy" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-05-19T12:00:58-07:00", + "abatementDateTime": "2010-05-26T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:c0be40f7-2570-4dfb-99a4-e151160140a3", + "resource": { + "resourceType": "Condition", + "id": "c0be40f7-2570-4dfb-99a4-e151160140a3", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" + }, + "dateRecorded": "2010-05-19", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + ], + "text": "Miscarriage in first trimester" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-05-19T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:7200df62-2767-4df5-8209-cd1c83527c78", + "resource": { + "resourceType": "Condition", + "id": "7200df62-2767-4df5-8209-cd1c83527c78", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" + }, + "dateRecorded": "2010-05-19", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "156073000", + "display": "Fetus with unknown complication" + } + ], + "text": "Fetus with unknown complication" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-05-19T12:00:58-07:00", + "abatementDateTime": "2010-05-26T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f6960f74-9a0c-4f10-9da6-b01282cdb84e", + "resource": { + "resourceType": "Procedure", + "id": "f6960f74-9a0c-4f10-9da6-b01282cdb84e", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "252160004", + "display": "Standard pregnancy test" + } + ], + "text": "Standard pregnancy test" + }, + "reasonReference": { + "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" + }, + "performedPeriod": { + "start": "2010-05-19T12:00:58-07:00", + "end": "2010-05-19T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:505b8a61-94ef-4a73-a8c3-c5212c97026d", + "resource": { + "resourceType": "Procedure", + "id": "505b8a61-94ef-4a73-a8c3-c5212c97026d", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "169230002", + "display": "Ultrasound scan for fetal viability" + } + ], + "text": "Ultrasound scan for fetal viability" + }, + "reasonReference": { + "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" + }, + "performedPeriod": { + "start": "2010-05-19T12:00:58-07:00", + "end": "2010-05-19T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:cf50630e-4990-4ef1-b265-fd28fa193833", + "resource": { + "resourceType": "Claim", + "id": "cf50630e-4990-4ef1-b265-fd28fa193833", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + }, + { + "sequence": 2, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + }, + { + "sequence": 3, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "156073000", + "display": "Fetus with unknown complication" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "252160004", + "display": "Standard pregnancy test" + }, + "net": { + "value": 7287.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "169230002", + "display": "Ultrasound scan for fetal viability" + }, + "net": { + "value": 11149.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3", + "resource": { + "resourceType": "Encounter", + "id": "6dbc35fe-335f-41e3-8e8e-03472aacd1c3", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2010-05-26T12:00:58-07:00", + "end": "2010-05-26T12:45:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "156073000", + "display": "Fetus with unknown complication" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9158ff31-11f1-47af-a810-2d6bcbed613b", + "resource": { + "resourceType": "Procedure", + "id": "9158ff31-11f1-47af-a810-2d6bcbed613b", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination" + } + ], + "text": "Physical examination" + }, + "reasonReference": { + "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" + }, + "performedPeriod": { + "start": "2010-05-26T12:00:58-07:00", + "end": "2010-05-26T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2a5c8470-f88f-4a9a-864f-41896b228931", + "resource": { + "resourceType": "Procedure", + "id": "2a5c8470-f88f-4a9a-864f-41896b228931", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "171207006", + "display": "Depression screening" + } + ], + "text": "Depression screening" + }, + "reasonReference": { + "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" + }, + "performedPeriod": { + "start": "2010-05-26T12:00:58-07:00", + "end": "2010-05-26T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a9cc55fa-d313-466c-a27f-da7e9fb418c2", + "resource": { + "resourceType": "Claim", + "id": "a9cc55fa-d313-466c-a27f-da7e9fb418c2", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical examination" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "171207006", + "display": "Depression screening" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e96e887a-f63c-4d40-a800-2e529fb023b2", + "resource": { + "resourceType": "Encounter", + "id": "e96e887a-f63c-4d40-a800-2e529fb023b2", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2010-08-24T12:00:58-07:00", + "end": "2010-08-24T12:15:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d9faa764-02da-4f93-b161-e93fa8ddde6a", + "resource": { + "resourceType": "Condition", + "id": "d9faa764-02da-4f93-b161-e93fa8ddde6a", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:e96e887a-f63c-4d40-a800-2e529fb023b2" + }, + "dateRecorded": "2010-08-24", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-08-24T12:00:58-07:00", + "abatementDateTime": "2010-09-07T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:1ef2e389-18c3-406b-9538-7c0ea93a1a33", + "resource": { + "resourceType": "Claim", + "id": "1ef2e389-18c3-406b-9538-7c0ea93a1a33", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190", + "resource": { + "resourceType": "Encounter", + "id": "5e5de478-9759-429a-a50f-0d3960846190", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2011-06-29T12:00:58-07:00", + "end": "2011-06-29T12:15:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5d79d370-13a2-42e3-b1ec-c4e040642bb4", + "resource": { + "resourceType": "Observation", + "id": "5d79d370-13a2-42e3-b1ec-c4e040642bb4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:72746c63-ce6a-46f8-8e6a-7f6207fa433f", + "resource": { + "resourceType": "Observation", + "id": "72746c63-ce6a-46f8-8e6a-7f6207fa433f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c2e5d37-8ce6-4b1e-9b38-45f08a84bdb7", + "resource": { + "resourceType": "Observation", + "id": "4c2e5d37-8ce6-4b1e-9b38-45f08a84bdb7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:04c4da0c-07ae-4fd6-b803-a0b6cdfdf3e8", + "resource": { + "resourceType": "Observation", + "id": "04c4da0c-07ae-4fd6-b803-a0b6cdfdf3e8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf32795b-0510-43df-94e4-34036b0bbdd6", + "resource": { + "resourceType": "Observation", + "id": "cf32795b-0510-43df-94e4-34036b0bbdd6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 100, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a0cd6ca8-264f-43f2-a680-d381db876a71", + "resource": { + "resourceType": "Observation", + "id": "a0cd6ca8-264f-43f2-a680-d381db876a71", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c1771b6-a801-4dff-a054-9812dbdef71a", + "resource": { + "resourceType": "Observation", + "id": "9c1771b6-a801-4dff-a054-9812dbdef71a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6c66c74-2984-48f1-bed8-518e6f3e4878", + "resource": { + "resourceType": "Observation", + "id": "c6c66c74-2984-48f1-bed8-518e6f3e4878", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 194.99, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e0f29a6b-61ce-4b86-a741-2dc5046f7d83", + "resource": { + "resourceType": "Observation", + "id": "e0f29a6b-61ce-4b86-a741-2dc5046f7d83", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 104.79, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d241802-e659-4374-a695-bed1bbfb00c8", + "resource": { + "resourceType": "Observation", + "id": "0d241802-e659-4374-a695-bed1bbfb00c8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 110.85, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4e0f7d3-cef0-4b79-bb66-de7a9605794d", + "resource": { + "resourceType": "Observation", + "id": "c4e0f7d3-cef0-4b79-bb66-de7a9605794d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueQuantity": { + "value": 63.190, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2366fedc-da48-4d67-a8ec-67f4dd6f5f36", + "resource": { + "resourceType": "Observation", + "id": "2366fedc-da48-4d67-a8ec-67f4dd6f5f36", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac7e2e9b-0447-4a6e-8724-d3fc639f4a1e", + "resource": { + "resourceType": "Immunization", + "id": "ac7e2e9b-0447-4a6e-8724-d3fc639f4a1e", + "status": "completed", + "date": "2011-06-29T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f9075d2d-6a41-4b82-a158-76f5796a7ca0", + "resource": { + "resourceType": "Immunization", + "id": "f9075d2d-6a41-4b82-a158-76f5796a7ca0", + "status": "completed", + "date": "2011-06-29T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b5f5e6f7-0d4c-464a-9b2f-b6a7197cfb18", + "resource": { + "resourceType": "DiagnosticReport", + "id": "b5f5e6f7-0d4c-464a-9b2f-b6a7197cfb18", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" + }, + "effectiveDateTime": "2011-06-29T12:00:58-07:00", + "issued": "2011-06-29T12:00:58.308-07:00", + "performer": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "result": [ + { + "reference": "urn:uuid:c4e0f7d3-cef0-4b79-bb66-de7a9605794d", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:dbe7f641-9b9a-4f44-9355-aa1b8586d56e", + "resource": { + "resourceType": "Claim", + "id": "dbe7f641-9b9a-4f44-9355-aa1b8586d56e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9ee61fc6-9fe8-4616-a7bb-683ee034da6c", + "resource": { + "resourceType": "Encounter", + "id": "9ee61fc6-9fe8-4616-a7bb-683ee034da6c", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2011-06-29T12:00:58-07:00", + "end": "2011-06-29T12:53:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e55ecd48-77a1-4c51-aab1-14838de89810", + "resource": { + "resourceType": "Procedure", + "id": "e55ecd48-77a1-4c51-aab1-14838de89810", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "performedPeriod": { + "start": "2011-06-29T12:00:58-07:00", + "end": "2011-06-29T12:38:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:9ee61fc6-9fe8-4616-a7bb-683ee034da6c" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b268cef1-5fd9-4385-8223-75eb77400f6e", + "resource": { + "resourceType": "Claim", + "id": "b268cef1-5fd9-4385-8223-75eb77400f6e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + }, + "net": { + "value": 12853.05, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4", + "resource": { + "resourceType": "Encounter", + "id": "1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2012-06-15T12:00:58-07:00", + "end": "2012-06-15T12:15:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "201834006", + "display": "Localized, primary osteoarthritis of the hand" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68", + "resource": { + "resourceType": "Condition", + "id": "1dd1524c-d3bd-47d6-b99e-f51d62988a68", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" + }, + "dateRecorded": "2012-06-15", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "201834006", + "display": "Localized, primary osteoarthritis of the hand" + } + ], + "text": "Localized, primary osteoarthritis of the hand" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2012-06-15T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:305e9f59-1dc6-48d2-8c7a-77e4f4804555", + "resource": { + "resourceType": "Observation", + "id": "305e9f59-1dc6-48d2-8c7a-77e4f4804555", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" + }, + "effectiveDateTime": "2012-06-15T12:00:58-07:00", + "issued": "2012-06-15T12:00:58.308-07:00", + "valueQuantity": { + "value": 7, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6bd4f119-9abb-4a22-8f2f-ac781b6caa0b", + "resource": { + "resourceType": "MedicationOrder", + "id": "6bd4f119-9abb-4a22-8f2f-ac781b6caa0b", + "dateWritten": "2012-06-15T12:00:58-07:00", + "status": "active", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "prescriber": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + }, + "encounter": { + "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" + }, + "reasonReference": { + "reference": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "849574", + "display": "Naproxen sodium 220 MG Oral Tablet" + } + ], + "text": "Naproxen sodium 220 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:241322e6-6258-4f8f-bb9b-7db50ea1d724", + "resource": { + "resourceType": "Claim", + "id": "241322e6-6258-4f8f-bb9b-7db50ea1d724", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:6bd4f119-9abb-4a22-8f2f-ac781b6caa0b" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f842512a-bab8-4a84-b5ee-1326d9123818", + "resource": { + "resourceType": "CarePlan", + "id": "f842512a-bab8-4a84-b5ee-1326d9123818", + "text": { + "status": "generated", + "div": "
    Musculoskeletal care
    " + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "active", + "context": { + "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" + }, + "period": { + "start": "2012-06-15T12:00:58-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "408869004", + "display": "Musculoskeletal care" + } + ], + "text": "Musculoskeletal care" + } + ], + "addresses": [ + { + "reference": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "386294003", + "display": "Joint mobility exercises" + } + ], + "text": "Joint mobility exercises" + }, + "status": "in-progress", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266694003", + "display": "Heat therapy" + } + ], + "text": "Heat therapy" + }, + "status": "in-progress", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:d45e2e6e-10ed-4121-a429-9f4c15b74947", + "resource": { + "resourceType": "Claim", + "id": "d45e2e6e-10ed-4121-a429-9f4c15b74947", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "201834006", + "display": "Localized, primary osteoarthritis of the hand" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395", + "resource": { + "resourceType": "Encounter", + "id": "240e4ed8-ade1-44d5-86ef-4ea380cdf395", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2012-07-04T12:00:58-07:00", + "end": "2012-07-04T12:15:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:38f326e5-1dec-4a39-8995-b4064daf5f03", + "resource": { + "resourceType": "Observation", + "id": "38f326e5-1dec-4a39-8995-b4064daf5f03", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a8a88ac-bdb8-4a49-ad37-7a52131108ea", + "resource": { + "resourceType": "Observation", + "id": "5a8a88ac-bdb8-4a49-ad37-7a52131108ea", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:540fee28-8237-4115-86b4-562c107043d5", + "resource": { + "resourceType": "Observation", + "id": "540fee28-8237-4115-86b4-562c107043d5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd091d9d-c5d3-479e-9814-8afead7037e3", + "resource": { + "resourceType": "Observation", + "id": "dd091d9d-c5d3-479e-9814-8afead7037e3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d23a7ab-eede-4abb-aa68-b7f21ca5a322", + "resource": { + "resourceType": "Observation", + "id": "3d23a7ab-eede-4abb-aa68-b7f21ca5a322", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 71, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 137, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:218256f0-6bc4-46d8-9843-bbd0920737cf", + "resource": { + "resourceType": "Observation", + "id": "218256f0-6bc4-46d8-9843-bbd0920737cf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 99, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc6a4e2e-9bd3-469a-9197-1530712c795c", + "resource": { + "resourceType": "Observation", + "id": "fc6a4e2e-9bd3-469a-9197-1530712c795c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8593ac1e-1a4a-4672-aaae-0019a347b367", + "resource": { + "resourceType": "Observation", + "id": "8593ac1e-1a4a-4672-aaae-0019a347b367", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + }, + "effectiveDateTime": "2012-07-04T12:00:58-07:00", + "issued": "2012-07-04T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:02f0cbee-2bd9-4c49-9fc4-9e0c9463c9e0", + "resource": { + "resourceType": "Immunization", + "id": "02f0cbee-2bd9-4c49-9fc4-9e0c9463c9e0", + "status": "completed", + "date": "2012-07-04T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a652552c-9cc4-4354-825c-3e22e12f79e6", + "resource": { + "resourceType": "Immunization", + "id": "a652552c-9cc4-4354-825c-3e22e12f79e6", + "status": "completed", + "date": "2012-07-04T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:abb57b3a-9584-45fd-8152-4d6fb475fed8", + "resource": { + "resourceType": "Immunization", + "id": "abb57b3a-9584-45fd-8152-4d6fb475fed8", + "status": "completed", + "date": "2012-07-04T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:01360dba-1b28-43ed-af0a-d2754be4daa4", + "resource": { + "resourceType": "Claim", + "id": "01360dba-1b28-43ed-af0a-d2754be4daa4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634", + "resource": { + "resourceType": "Encounter", + "id": "083255a3-e7ff-47e7-b19f-12762d8c5634", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2013-07-10T12:00:58-07:00", + "end": "2013-07-10T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b4668fce-f765-4ec1-86ab-78a78f35473b", + "resource": { + "resourceType": "Observation", + "id": "b4668fce-f765-4ec1-86ab-78a78f35473b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:600defb2-666b-431d-a51c-0eec2aa8bb54", + "resource": { + "resourceType": "Observation", + "id": "600defb2-666b-431d-a51c-0eec2aa8bb54", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e6853fa-5ed1-45f6-9529-70b0e576def0", + "resource": { + "resourceType": "Observation", + "id": "3e6853fa-5ed1-45f6-9529-70b0e576def0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0721f1f3-f563-4c09-81d6-7af15c8b57c0", + "resource": { + "resourceType": "Observation", + "id": "0721f1f3-f563-4c09-81d6-7af15c8b57c0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aff7f7a1-502d-4e63-a1ee-1f9fec6b2e3a", + "resource": { + "resourceType": "Observation", + "id": "aff7f7a1-502d-4e63-a1ee-1f9fec6b2e3a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 90, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 109, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f2e4a481-2fe2-4b7d-85a5-adabfea77016", + "resource": { + "resourceType": "Observation", + "id": "f2e4a481-2fe2-4b7d-85a5-adabfea77016", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 95, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f7155a03-508f-4960-8138-4eeb444b5930", + "resource": { + "resourceType": "Observation", + "id": "f7155a03-508f-4960-8138-4eeb444b5930", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76b8c2c5-c04b-45a6-aa27-b805c92b286d", + "resource": { + "resourceType": "Observation", + "id": "76b8c2c5-c04b-45a6-aa27-b805c92b286d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 7.6507, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4bc7ed46-4ae8-4675-8947-ab9a54702eb4", + "resource": { + "resourceType": "Observation", + "id": "4bc7ed46-4ae8-4675-8947-ab9a54702eb4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 4.1549, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3633b3f9-8a85-40ff-ab55-6af7bf2b12c1", + "resource": { + "resourceType": "Observation", + "id": "3633b3f9-8a85-40ff-ab55-6af7bf2b12c1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 12.313, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db3d344a-fd48-42a0-b9a5-6f795324d1af", + "resource": { + "resourceType": "Observation", + "id": "db3d344a-fd48-42a0-b9a5-6f795324d1af", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 39.705, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1eb8ca1-7770-4829-b9c1-596846764e0f", + "resource": { + "resourceType": "Observation", + "id": "a1eb8ca1-7770-4829-b9c1-596846764e0f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 89.137, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bad012c-757e-415e-985f-9d682c677544", + "resource": { + "resourceType": "Observation", + "id": "7bad012c-757e-415e-985f-9d682c677544", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.554, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25c86423-01b5-438d-b1f1-89d3b68794ea", + "resource": { + "resourceType": "Observation", + "id": "25c86423-01b5-438d-b1f1-89d3b68794ea", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 34.551, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddc16784-c507-4108-a6d8-cae0b05ba254", + "resource": { + "resourceType": "Observation", + "id": "ddc16784-c507-4108-a6d8-cae0b05ba254", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 42.072, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db9f82bd-802d-499c-8217-0169a0fe0339", + "resource": { + "resourceType": "Observation", + "id": "db9f82bd-802d-499c-8217-0169a0fe0339", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 384.17, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:788de2cc-46d5-4303-a720-31c5555b0578", + "resource": { + "resourceType": "Observation", + "id": "788de2cc-46d5-4303-a720-31c5555b0578", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 514.21, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76cd5814-68d0-4675-8792-418d633702f7", + "resource": { + "resourceType": "Observation", + "id": "76cd5814-68d0-4675-8792-418d633702f7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueQuantity": { + "value": 11.796, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a037d03b-3b1d-439e-b16f-295b9f31e79d", + "resource": { + "resourceType": "Observation", + "id": "a037d03b-3b1d-439e-b16f-295b9f31e79d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2a86e052-4200-499e-99e7-48f56bea808d", + "resource": { + "resourceType": "Procedure", + "id": "2a86e052-4200-499e-99e7-48f56bea808d", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2013-07-10T12:00:58-07:00", + "end": "2013-07-10T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:6dfdb7aa-3bd7-41db-96e1-5017516c6bc5", + "resource": { + "resourceType": "Immunization", + "id": "6dfdb7aa-3bd7-41db-96e1-5017516c6bc5", + "status": "completed", + "date": "2013-07-10T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:41cf5440-11c5-4b5c-9026-0b6a762f7afe", + "resource": { + "resourceType": "DiagnosticReport", + "id": "41cf5440-11c5-4b5c-9026-0b6a762f7afe", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" + }, + "effectiveDateTime": "2013-07-10T12:00:58-07:00", + "issued": "2013-07-10T12:00:58.308-07:00", + "performer": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "result": [ + { + "reference": "urn:uuid:76cd5814-68d0-4675-8792-418d633702f7", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:4ebd6277-499d-4ad5-baf8-b65379a59708", + "resource": { + "resourceType": "Claim", + "id": "4ebd6277-499d-4ad5-baf8-b65379a59708", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 584.84, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e", + "resource": { + "resourceType": "Encounter", + "id": "79e96930-9f24-446f-b952-9183dc2f983e", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2014-01-24T11:00:58-08:00", + "end": "2014-01-24T11:15:58-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8f9bf7af-1a0b-4c97-9237-6894444ab1c5", + "resource": { + "resourceType": "Condition", + "id": "8f9bf7af-1a0b-4c97-9237-6894444ab1c5", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e" + }, + "dateRecorded": "2014-01-24", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2014-01-24T11:00:58-08:00", + "abatementDateTime": "2014-02-05T11:00:58-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:580e4c5e-240f-4160-a5ee-fdd3c10b3700", + "resource": { + "resourceType": "Observation", + "id": "580e4c5e-240f-4160-a5ee-fdd3c10b3700", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e" + }, + "effectiveDateTime": "2014-01-24T11:00:58-08:00", + "issued": "2014-01-24T11:00:58.308-08:00", + "valueQuantity": { + "value": 37.233, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ae86b3f9-9e0e-42d1-909c-6576b417868f", + "resource": { + "resourceType": "Claim", + "id": "ae86b3f9-9e0e-42d1-909c-6576b417868f", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13", + "resource": { + "resourceType": "Encounter", + "id": "3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2014-07-16T12:00:58-07:00", + "end": "2014-07-16T12:15:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7fe14f9b-06ba-4484-bc45-69b00e554b21", + "resource": { + "resourceType": "Observation", + "id": "7fe14f9b-06ba-4484-bc45-69b00e554b21", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9996ad06-6867-4b1b-ab59-63dc92605a2f", + "resource": { + "resourceType": "Observation", + "id": "9996ad06-6867-4b1b-ab59-63dc92605a2f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7adc1ace-11ff-40bc-9022-aed720fa2fdf", + "resource": { + "resourceType": "Observation", + "id": "7adc1ace-11ff-40bc-9022-aed720fa2fdf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f42104dc-7a64-4a17-90f0-a5513734b62a", + "resource": { + "resourceType": "Observation", + "id": "f42104dc-7a64-4a17-90f0-a5513734b62a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71a86a92-d879-4377-9008-ec91d062de7d", + "resource": { + "resourceType": "Observation", + "id": "71a86a92-d879-4377-9008-ec91d062de7d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 137, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:370be409-7013-42cb-9735-f4e85653c888", + "resource": { + "resourceType": "Observation", + "id": "370be409-7013-42cb-9735-f4e85653c888", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 87, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ad14f7d9-d641-4b85-a4be-3a9f0c666440", + "resource": { + "resourceType": "Observation", + "id": "ad14f7d9-d641-4b85-a4be-3a9f0c666440", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e081309e-3f38-4965-8f1e-bdc8ee35cfe8", + "resource": { + "resourceType": "Observation", + "id": "e081309e-3f38-4965-8f1e-bdc8ee35cfe8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 164.71, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e60fb867-b6d9-43f6-8b8d-6bfa67b82efd", + "resource": { + "resourceType": "Observation", + "id": "e60fb867-b6d9-43f6-8b8d-6bfa67b82efd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 134.36, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:325e8e02-fde0-45b1-b89b-0842612852db", + "resource": { + "resourceType": "Observation", + "id": "325e8e02-fde0-45b1-b89b-0842612852db", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 58.020, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a117c30-470c-42e9-83fd-751207b99985", + "resource": { + "resourceType": "Observation", + "id": "3a117c30-470c-42e9-83fd-751207b99985", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueQuantity": { + "value": 79.810, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:00a7ebab-6270-43b4-b31a-82bc49456473", + "resource": { + "resourceType": "Observation", + "id": "00a7ebab-6270-43b4-b31a-82bc49456473", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:515bb603-b614-433d-92ab-a33aa592a49f", + "resource": { + "resourceType": "Immunization", + "id": "515bb603-b614-433d-92ab-a33aa592a49f", + "status": "completed", + "date": "2014-07-16T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d2ee14f9-fe75-4b21-b3b6-d82e3ff156f9", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d2ee14f9-fe75-4b21-b3b6-d82e3ff156f9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" + }, + "effectiveDateTime": "2014-07-16T12:00:58-07:00", + "issued": "2014-07-16T12:00:58.308-07:00", + "performer": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "result": [ + { + "reference": "urn:uuid:3a117c30-470c-42e9-83fd-751207b99985", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7ae5ea9e-1779-4946-ae63-b91341a2eaa0", + "resource": { + "resourceType": "Claim", + "id": "7ae5ea9e-1779-4946-ae63-b91341a2eaa0", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a42d3289-5a4b-426f-95d7-2ebae5910f85", + "resource": { + "resourceType": "Encounter", + "id": "a42d3289-5a4b-426f-95d7-2ebae5910f85", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2014-07-24T12:00:58-07:00", + "end": "2014-07-24T12:15:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4106bd3e-bcf0-4870-bb18-ed13760df7a4", + "resource": { + "resourceType": "Condition", + "id": "4106bd3e-bcf0-4870-bb18-ed13760df7a4", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:a42d3289-5a4b-426f-95d7-2ebae5910f85" + }, + "dateRecorded": "2014-07-24", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2014-07-24T12:00:58-07:00", + "abatementDateTime": "2014-07-31T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f4153721-c628-445e-ad24-31c30916f66c", + "resource": { + "resourceType": "Claim", + "id": "f4153721-c628-445e-ad24-31c30916f66c", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7", + "resource": { + "resourceType": "Encounter", + "id": "3d768562-15a0-4a0f-b285-b9dae7e085f7", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2015-07-22T12:00:58-07:00", + "end": "2015-07-22T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e54762e7-1536-4237-addf-293f9ac683b3", + "resource": { + "resourceType": "Observation", + "id": "e54762e7-1536-4237-addf-293f9ac683b3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36ce9719-9fb3-46f6-91dd-ccf5a53a5f04", + "resource": { + "resourceType": "Observation", + "id": "36ce9719-9fb3-46f6-91dd-ccf5a53a5f04", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db948571-13c2-444a-98a9-cd18dc2bb9ae", + "resource": { + "resourceType": "Observation", + "id": "db948571-13c2-444a-98a9-cd18dc2bb9ae", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d0ac5208-5945-4e36-ac28-9ac4b0dae76b", + "resource": { + "resourceType": "Observation", + "id": "d0ac5208-5945-4e36-ac28-9ac4b0dae76b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6fe412e7-3a60-46fd-a173-9006ab260614", + "resource": { + "resourceType": "Observation", + "id": "6fe412e7-3a60-46fd-a173-9006ab260614", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a32447e4-79cc-45f9-9798-c650dafd469b", + "resource": { + "resourceType": "Observation", + "id": "a32447e4-79cc-45f9-9798-c650dafd469b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 100, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1aef7b5-9a41-4028-95cd-070ecda7c4ad", + "resource": { + "resourceType": "Observation", + "id": "f1aef7b5-9a41-4028-95cd-070ecda7c4ad", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:434f732e-1a60-4b46-a4af-f16bb1831bbf", + "resource": { + "resourceType": "Observation", + "id": "434f732e-1a60-4b46-a4af-f16bb1831bbf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + }, + "effectiveDateTime": "2015-07-22T12:00:58-07:00", + "issued": "2015-07-22T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a8e2d0d4-b7a8-460a-8113-cff272adc57b", + "resource": { + "resourceType": "Procedure", + "id": "a8e2d0d4-b7a8-460a-8113-cff272adc57b", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2015-07-22T12:00:58-07:00", + "end": "2015-07-22T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5ff95e5d-e7bb-4bfd-9596-80aaef6c0d2b", + "resource": { + "resourceType": "Immunization", + "id": "5ff95e5d-e7bb-4bfd-9596-80aaef6c0d2b", + "status": "completed", + "date": "2015-07-22T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4639e7e4-1217-4adc-904f-36a0de124546", + "resource": { + "resourceType": "Claim", + "id": "4639e7e4-1217-4adc-904f-36a0de124546", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 396.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:567cba0f-6094-40f3-997e-85ae2e6c810b", + "resource": { + "resourceType": "Encounter", + "id": "567cba0f-6094-40f3-997e-85ae2e6c810b", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2016-06-27T12:00:58-07:00", + "end": "2016-06-27T12:40:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6cd1c759-73c5-4293-830f-70d2f6aaf262", + "resource": { + "resourceType": "Procedure", + "id": "6cd1c759-73c5-4293-830f-70d2f6aaf262", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "performedPeriod": { + "start": "2016-06-27T12:00:58-07:00", + "end": "2016-06-27T12:25:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:567cba0f-6094-40f3-997e-85ae2e6c810b" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c4bb412b-a76e-4270-ad57-2ff178f77ac3", + "resource": { + "resourceType": "Claim", + "id": "c4bb412b-a76e-4270-ad57-2ff178f77ac3", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + }, + "net": { + "value": 16189.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99", + "resource": { + "resourceType": "Encounter", + "id": "c8931c16-5d87-4d20-83bd-65e1e90fbd99", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2016-07-27T12:00:58-07:00", + "end": "2016-07-27T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f9eba7ef-b8d9-4d5b-bd54-3e5d34d18b1b", + "resource": { + "resourceType": "Observation", + "id": "f9eba7ef-b8d9-4d5b-bd54-3e5d34d18b1b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b9e8a11-50d9-4d70-b635-c8646b15b444", + "resource": { + "resourceType": "Observation", + "id": "1b9e8a11-50d9-4d70-b635-c8646b15b444", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ae5e618d-9ed5-4a8f-8812-562a422e9dc3", + "resource": { + "resourceType": "Observation", + "id": "ae5e618d-9ed5-4a8f-8812-562a422e9dc3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:745edee9-abef-4ab6-bd47-e50b9e0e96bc", + "resource": { + "resourceType": "Observation", + "id": "745edee9-abef-4ab6-bd47-e50b9e0e96bc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6f8d163-9560-4696-875e-222bc5d9599f", + "resource": { + "resourceType": "Observation", + "id": "e6f8d163-9560-4696-875e-222bc5d9599f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 107, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebf6d0e7-e45c-4109-a2aa-c0d9e43b03b0", + "resource": { + "resourceType": "Observation", + "id": "ebf6d0e7-e45c-4109-a2aa-c0d9e43b03b0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 70, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9fc136c1-6ae9-4c6b-b429-0f228a7bf488", + "resource": { + "resourceType": "Observation", + "id": "9fc136c1-6ae9-4c6b-b429-0f228a7bf488", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8f7b5bdf-4659-49ca-a58a-89b90869da05", + "resource": { + "resourceType": "Observation", + "id": "8f7b5bdf-4659-49ca-a58a-89b90869da05", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + }, + "effectiveDateTime": "2016-07-27T12:00:58-07:00", + "issued": "2016-07-27T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e3a78a6-3c18-45fe-9988-c6517a14697c", + "resource": { + "resourceType": "Procedure", + "id": "5e3a78a6-3c18-45fe-9988-c6517a14697c", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2016-07-27T12:00:58-07:00", + "end": "2016-07-27T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e57ffcef-b89a-4b77-8bd0-90245c77b94e", + "resource": { + "resourceType": "Immunization", + "id": "e57ffcef-b89a-4b77-8bd0-90245c77b94e", + "status": "completed", + "date": "2016-07-27T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:cb1ebce3-7b75-40dd-94e7-91ca42903108", + "resource": { + "resourceType": "Claim", + "id": "cb1ebce3-7b75-40dd-94e7-91ca42903108", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 647.18, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39", + "resource": { + "resourceType": "Encounter", + "id": "4a5a04f3-96ae-4122-9a06-48e5a6af8a39", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2017-08-02T12:00:58-07:00", + "end": "2017-08-02T12:15:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:24daf66a-4e5b-4ed4-a2ad-e681bab296fd", + "resource": { + "resourceType": "Observation", + "id": "24daf66a-4e5b-4ed4-a2ad-e681bab296fd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e494d610-29fc-4aa5-80a3-2d17c5070303", + "resource": { + "resourceType": "Observation", + "id": "e494d610-29fc-4aa5-80a3-2d17c5070303", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ab1a0f5-97fb-4774-827b-2e2ef9ff202b", + "resource": { + "resourceType": "Observation", + "id": "9ab1a0f5-97fb-4774-827b-2e2ef9ff202b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b53552c4-019f-433f-a611-81138162a747", + "resource": { + "resourceType": "Observation", + "id": "b53552c4-019f-433f-a611-81138162a747", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bb9ee043-e1d4-4f14-a95e-2d39337b60f1", + "resource": { + "resourceType": "Observation", + "id": "bb9ee043-e1d4-4f14-a95e-2d39337b60f1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0640cdf1-ef7b-42f8-8d0d-3689a7cc8e78", + "resource": { + "resourceType": "Observation", + "id": "0640cdf1-ef7b-42f8-8d0d-3689a7cc8e78", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 78, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:440a4598-a8ff-43ac-a7a9-d5f99c4b56cb", + "resource": { + "resourceType": "Observation", + "id": "440a4598-a8ff-43ac-a7a9-d5f99c4b56cb", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0216a17a-f1d8-4c76-884a-8a2183878030", + "resource": { + "resourceType": "Observation", + "id": "0216a17a-f1d8-4c76-884a-8a2183878030", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 163.89, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:10fae0a3-57f5-408e-967b-3c6a77304419", + "resource": { + "resourceType": "Observation", + "id": "10fae0a3-57f5-408e-967b-3c6a77304419", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 116.74, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3669070-1cc6-48af-9029-f23b3a690767", + "resource": { + "resourceType": "Observation", + "id": "e3669070-1cc6-48af-9029-f23b3a690767", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 71.120, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7c490c0-238b-4f3c-a6e3-e796a356ad3e", + "resource": { + "resourceType": "Observation", + "id": "b7c490c0-238b-4f3c-a6e3-e796a356ad3e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueQuantity": { + "value": 69.430, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ce286d4-a2df-4f98-ae0c-14db797963d9", + "resource": { + "resourceType": "Observation", + "id": "5ce286d4-a2df-4f98-ae0c-14db797963d9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:79557d6a-cdc0-4988-9fd7-2df640bdbc16", + "resource": { + "resourceType": "Immunization", + "id": "79557d6a-cdc0-4988-9fd7-2df640bdbc16", + "status": "completed", + "date": "2017-08-02T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1f9c7067-6385-4093-8a47-eb796fc74561", + "resource": { + "resourceType": "DiagnosticReport", + "id": "1f9c7067-6385-4093-8a47-eb796fc74561", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" + }, + "effectiveDateTime": "2017-08-02T12:00:58-07:00", + "issued": "2017-08-02T12:00:58.308-07:00", + "performer": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "result": [ + { + "reference": "urn:uuid:b7c490c0-238b-4f3c-a6e3-e796a356ad3e", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:2de52b8d-448f-4d64-ba18-dd8c5a167a3e", + "resource": { + "resourceType": "Claim", + "id": "2de52b8d-448f-4d64-ba18-dd8c5a167a3e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635", + "resource": { + "resourceType": "Encounter", + "id": "68ea2cc6-d2b0-460c-9724-1b8e9fc17635", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + } + } + ], + "period": { + "start": "2018-04-18T12:00:58-07:00", + "end": "2018-04-18T12:30:58-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b", + "resource": { + "resourceType": "Condition", + "id": "333435c4-b72c-4f42-a672-8861806c7c5b", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" + }, + "dateRecorded": "2018-04-18", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2018-04-18T12:00:58-07:00", + "abatementDateTime": "2018-04-25T12:00:58-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:aaff4b8b-ab0a-4941-be2a-b093e29eef8e", + "resource": { + "resourceType": "Observation", + "id": "aaff4b8b-ab0a-4941-be2a-b093e29eef8e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" + }, + "effectiveDateTime": "2018-04-18T12:00:58-07:00", + "issued": "2018-04-18T12:00:58.308-07:00", + "valueQuantity": { + "value": 38.525, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ad28370-2ab5-42e7-896c-e5bacc7fbc79", + "resource": { + "resourceType": "Procedure", + "id": "5ad28370-2ab5-42e7-896c-e5bacc7fbc79", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "reasonReference": { + "reference": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b" + }, + "performedPeriod": { + "start": "2018-04-18T12:00:58-07:00", + "end": "2018-04-18T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:8382a2bb-e319-40eb-ab05-98a73700a74f", + "resource": { + "resourceType": "MedicationOrder", + "id": "8382a2bb-e319-40eb-ab05-98a73700a74f", + "dateWritten": "2018-04-18T12:00:58-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "prescriber": { + "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" + }, + "encounter": { + "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" + }, + "reasonReference": { + "reference": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "834102", + "display": "Penicillin V Potassium 500 MG Oral Tablet" + } + ], + "text": "Penicillin V Potassium 500 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:db44cd30-b644-41b1-ad33-da4655ed8912", + "resource": { + "resourceType": "Claim", + "id": "db44cd30-b644-41b1-ad33-da4655ed8912", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:8382a2bb-e319-40eb-ab05-98a73700a74f" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d296fe0d-9770-4382-a2c3-814a59e19249", + "resource": { + "resourceType": "Claim", + "id": "d296fe0d-9770-4382-a2c3-814a59e19249", + "type": "institutional", + "organization": { + "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + }, + "net": { + "value": 3449.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be", + "resource": { + "resourceType": "Encounter", + "id": "ba203c7a-efb5-4008-ba3e-12aacf75b7be", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2018-08-08T12:00:58-07:00", + "end": "2018-08-08T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c11057bb-b0d8-4e39-84e6-0f0d3ef5da44", + "resource": { + "resourceType": "Observation", + "id": "c11057bb-b0d8-4e39-84e6-0f0d3ef5da44", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e768f70e-bf68-40ff-bd42-ef89773fca31", + "resource": { + "resourceType": "Observation", + "id": "e768f70e-bf68-40ff-bd42-ef89773fca31", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bd79284-fed8-47b1-9add-e51d61aca0fc", + "resource": { + "resourceType": "Observation", + "id": "7bd79284-fed8-47b1-9add-e51d61aca0fc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c65c5c6-e61b-4087-abab-2996a5ab4802", + "resource": { + "resourceType": "Observation", + "id": "9c65c5c6-e61b-4087-abab-2996a5ab4802", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ad76fead-9f21-4bc2-9c9d-3bdd08cdd4ef", + "resource": { + "resourceType": "Observation", + "id": "ad76fead-9f21-4bc2-9c9d-3bdd08cdd4ef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 89, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 112, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d40ab67c-b7f7-4d1c-a09c-5214286ddf03", + "resource": { + "resourceType": "Observation", + "id": "d40ab67c-b7f7-4d1c-a09c-5214286ddf03", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 88, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:910e9488-2c61-4290-97ca-e3205c9f32f1", + "resource": { + "resourceType": "Observation", + "id": "910e9488-2c61-4290-97ca-e3205c9f32f1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:daa89c5e-116d-4445-9ce6-586bc5551edb", + "resource": { + "resourceType": "Observation", + "id": "daa89c5e-116d-4445-9ce6-586bc5551edb", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 3.8248, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:de2d496b-efff-4706-b4ee-798a7e91b01b", + "resource": { + "resourceType": "Observation", + "id": "de2d496b-efff-4706-b4ee-798a7e91b01b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 4.3428, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ae49c09-8014-48b5-902b-b03e1d1e184d", + "resource": { + "resourceType": "Observation", + "id": "1ae49c09-8014-48b5-902b-b03e1d1e184d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 12.671, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d29d9dce-74eb-43c5-85a2-b9788dd25854", + "resource": { + "resourceType": "Observation", + "id": "d29d9dce-74eb-43c5-85a2-b9788dd25854", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 46.024, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1531a503-62f1-4cca-95d2-f4bf159ac353", + "resource": { + "resourceType": "Observation", + "id": "1531a503-62f1-4cca-95d2-f4bf159ac353", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 83.500, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a2546a4-f65b-4239-b2f7-f453ca6e6ade", + "resource": { + "resourceType": "Observation", + "id": "3a2546a4-f65b-4239-b2f7-f453ca6e6ade", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 30.750, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51c606b1-6fab-4804-bcd7-c3375755ad6a", + "resource": { + "resourceType": "Observation", + "id": "51c606b1-6fab-4804-bcd7-c3375755ad6a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 34.274, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:18bab09d-0472-4686-aeba-a9c2783666a4", + "resource": { + "resourceType": "Observation", + "id": "18bab09d-0472-4686-aeba-a9c2783666a4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 43.191, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ccefee54-1a5a-40ea-a83f-c9b5ff5ad92e", + "resource": { + "resourceType": "Observation", + "id": "ccefee54-1a5a-40ea-a83f-c9b5ff5ad92e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 312.26, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4f3f1787-3c93-4b8a-84c5-62ef01d4873e", + "resource": { + "resourceType": "Observation", + "id": "4f3f1787-3c93-4b8a-84c5-62ef01d4873e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 436.34, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:42965d15-3301-4c48-8d25-738b68e01dd2", + "resource": { + "resourceType": "Observation", + "id": "42965d15-3301-4c48-8d25-738b68e01dd2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueQuantity": { + "value": 12.100, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:082fb329-33f0-495f-b84f-41ac859eb949", + "resource": { + "resourceType": "Observation", + "id": "082fb329-33f0-495f-b84f-41ac859eb949", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:be6662e1-fa6d-4128-a972-7fc0954f94a4", + "resource": { + "resourceType": "Procedure", + "id": "be6662e1-fa6d-4128-a972-7fc0954f94a4", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2018-08-08T12:00:58-07:00", + "end": "2018-08-08T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9d87a700-2cf6-4f23-a413-f8bb7f663675", + "resource": { + "resourceType": "Immunization", + "id": "9d87a700-2cf6-4f23-a413-f8bb7f663675", + "status": "completed", + "date": "2018-08-08T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f091e585-428c-4782-a5da-2968701c1465", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f091e585-428c-4782-a5da-2968701c1465", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" + }, + "effectiveDateTime": "2018-08-08T12:00:58-07:00", + "issued": "2018-08-08T12:00:58.308-07:00", + "performer": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "result": [ + { + "reference": "urn:uuid:42965d15-3301-4c48-8d25-738b68e01dd2", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:2b30f498-2f90-437c-98f4-bef44474b923", + "resource": { + "resourceType": "Claim", + "id": "2b30f498-2f90-437c-98f4-bef44474b923", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 565.66, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821", + "resource": { + "resourceType": "Encounter", + "id": "3a3a2573-b9b4-4889-bca7-741859f50821", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" + } + } + ], + "period": { + "start": "2019-08-14T12:00:58-07:00", + "end": "2019-08-14T12:30:58-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ab6bc4b0-b00a-46ea-8b1c-d5a705984aec", + "resource": { + "resourceType": "Observation", + "id": "ab6bc4b0-b00a-46ea-8b1c-d5a705984aec", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 167.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ce77c79-b222-4514-aa61-502863f98fa7", + "resource": { + "resourceType": "Observation", + "id": "1ce77c79-b222-4514-aa61-502863f98fa7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f40c57fd-3980-4812-8717-f7485e845269", + "resource": { + "resourceType": "Observation", + "id": "f40c57fd-3980-4812-8717-f7485e845269", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 78.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e063568-0f08-44d6-86b7-049deac8d2e2", + "resource": { + "resourceType": "Observation", + "id": "1e063568-0f08-44d6-86b7-049deac8d2e2", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 27.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:13b4ec52-110c-4f69-8de7-fc748c7f878b", + "resource": { + "resourceType": "Observation", + "id": "13b4ec52-110c-4f69-8de7-fc748c7f878b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 123, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebf25abe-a307-4b91-a440-e5e79287a97a", + "resource": { + "resourceType": "Observation", + "id": "ebf25abe-a307-4b91-a440-e5e79287a97a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a095c10f-4697-4b7d-ac78-db87fc120746", + "resource": { + "resourceType": "Observation", + "id": "a095c10f-4697-4b7d-ac78-db87fc120746", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eb5cfa3d-6f2b-46d6-b8c0-dccd54a1d296", + "resource": { + "resourceType": "Observation", + "id": "eb5cfa3d-6f2b-46d6-b8c0-dccd54a1d296", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + }, + "effectiveDateTime": "2019-08-14T12:00:58-07:00", + "issued": "2019-08-14T12:00:58.308-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f18047c-a076-4cba-a2d7-90027a3f2254", + "resource": { + "resourceType": "Procedure", + "id": "5f18047c-a076-4cba-a2d7-90027a3f2254", + "subject": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2019-08-14T12:00:58-07:00", + "end": "2019-08-14T12:15:58-07:00" + }, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:fce25c30-850e-481a-a82a-6e6c1220d2a1", + "resource": { + "resourceType": "Immunization", + "id": "fce25c30-850e-481a-a82a-6e6c1220d2a1", + "status": "completed", + "date": "2019-08-14T12:00:58-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:543426ea-0bfc-4402-8f95-ccd34b7b6ddd", + "resource": { + "resourceType": "Claim", + "id": "543426ea-0bfc-4402-8f95-ccd34b7b6ddd", + "type": "institutional", + "organization": { + "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 619.62, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json new file mode 100644 index 000000000000..3a6ae7a6d914 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json @@ -0,0 +1,10047 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f", + "resource": { + "resourceType": "Patient", + "id": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7196695550560093073 Population seed: 1586309754086
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Race", + "code": "2106-3", + "display": "White" + } + ], + "text": "White" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/Ethnicity", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + ], + "text": "Not Hispanic or Latino" + } + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Daniele610 Zemlak964" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "M" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Ashburnham", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.6685442920261035 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 48.331455707973895 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR" + } + ] + }, + "system": "http://hospital.smarthealthit.org", + "value": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB" + } + ] + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-11-6709" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "DL" + } + ] + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99940409" + } + ], + "name": [ + { + "use": "official", + "family": [ + "Hintz995" + ], + "given": [ + "Terry864" + ], + "prefix": [ + "Mr." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-451-4282", + "use": "home" + } + ], + "gender": "male", + "birthDate": "1970-06-15", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.305010719394204 + }, + { + "url": "longitude", + "valueDecimal": -72.70463521406347 + } + ] + } + ], + "line": [ + "718 Toy Parade" + ], + "city": "Easthampton", + "state": "Massachusetts", + "postalCode": "01027", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "M" + } + ] + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", + "resource": { + "resourceType": "Organization", + "id": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "PCP15859", + "telecom": [ + { + "system": "phone", + "value": "413-387-2118" + } + ], + "address": [ + { + "line": [ + "92 MAIN ST" + ], + "city": "FLORENCE", + "state": "MA", + "postalCode": "01062-1460", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd", + "resource": { + "resourceType": "Practitioner", + "id": "6d5162ad-3294-398e-9eb4-7021465d4bcd", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10930" + } + ], + "active": true, + "name": { + "family": [ + "Pfeffer420" + ], + "given": [ + "Darryl392" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "92 MAIN ST" + ], + "city": "FLORENCE", + "state": "MA", + "postalCode": "01062-1460", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:2a908737-0281-49dc-be45-550d71d74d25", + "resource": { + "resourceType": "Encounter", + "id": "2a908737-0281-49dc-be45-550d71d74d25", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "1998-08-24T15:14:32-07:00", + "end": "1998-08-24T15:44:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e494cb27-116c-46ed-9ed5-250cfb3c86e6", + "resource": { + "resourceType": "Condition", + "id": "e494cb27-116c-46ed-9ed5-250cfb3c86e6", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:2a908737-0281-49dc-be45-550d71d74d25" + }, + "dateRecorded": "1998-08-24", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "1998-08-24T15:14:32-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b0433c04-7930-4449-8442-e6523198d4bc", + "resource": { + "resourceType": "Claim", + "id": "b0433c04-7930-4449-8442-e6523198d4bc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12", + "resource": { + "resourceType": "Organization", + "id": "49318f80-bd8b-3fc7-a096-ac43088b0c12", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "COOLEY DICKINSON HOSPITAL INC,THE", + "telecom": [ + { + "system": "phone", + "value": "4135822000" + } + ], + "address": [ + { + "line": [ + "30 LOCUST STREET" + ], + "city": "NORTHAMPTON", + "state": "MA", + "postalCode": "01060", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7", + "resource": { + "resourceType": "Practitioner", + "id": "c1a3f738-c767-30ab-951c-5c60f3c569c7", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "60" + } + ], + "active": true, + "name": { + "family": [ + "Hermiston71" + ], + "given": [ + "Andrea7" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "30 LOCUST STREET" + ], + "city": "NORTHAMPTON", + "state": "MA", + "postalCode": "01060", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:0de358a5-2910-44b5-8b86-2154eb0e0fba", + "resource": { + "resourceType": "Encounter", + "id": "0de358a5-2910-44b5-8b86-2154eb0e0fba", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2001-04-16T15:14:32-07:00", + "end": "2012-04-30T15:14:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c734c612-92d9-4ba1-9c16-0e59dcde5f25", + "resource": { + "resourceType": "Claim", + "id": "c734c612-92d9-4ba1-9c16-0e59dcde5f25", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969", + "resource": { + "resourceType": "Encounter", + "id": "a4b4fba9-cf37-4c74-91f3-e820bf402969", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2004-08-30T15:14:32-07:00", + "end": "2004-08-30T15:29:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a3a299cc-81e0-4637-ad2f-dc392046e5dd", + "resource": { + "resourceType": "Condition", + "id": "a3a299cc-81e0-4637-ad2f-dc392046e5dd", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" + }, + "dateRecorded": "2004-08-30", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2004-08-30T15:14:32-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:bddfe782-62ba-494b-a368-bf70abb1a0c3", + "resource": { + "resourceType": "Condition", + "id": "bddfe782-62ba-494b-a368-bf70abb1a0c3", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" + }, + "dateRecorded": "2004-08-30", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "onsetDateTime": "2004-08-30T15:14:32-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:018ae8c8-863f-4b7a-b101-15966d2d926f", + "resource": { + "resourceType": "CarePlan", + "id": "018ae8c8-863f-4b7a-b101-15966d2d926f", + "text": { + "status": "generated", + "div": "
    Diabetes self management plan
    " + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "active", + "context": { + "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" + }, + "period": { + "start": "2004-08-30T15:14:32-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698360004", + "display": "Diabetes self management plan" + } + ], + "text": "Diabetes self management plan" + } + ], + "addresses": [ + { + "reference": "urn:uuid:a3a299cc-81e0-4637-ad2f-dc392046e5dd" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "160670007", + "display": "Diabetic diet" + } + ], + "text": "Diabetic diet" + }, + "status": "in-progress", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:e0f3e870-fdb7-45e3-ad22-e13879ee9743", + "resource": { + "resourceType": "Claim", + "id": "e0f3e870-fdb7-45e3-ad22-e13879ee9743", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + }, + { + "sequence": 2, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:58764f28-1efe-463c-b978-747589b6978b", + "resource": { + "resourceType": "Encounter", + "id": "58764f28-1efe-463c-b978-747589b6978b", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2004-08-30T15:14:32-07:00", + "end": "2004-08-30T16:36:32-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:00347e04-8907-43f6-a90d-03c8af72cd5c", + "resource": { + "resourceType": "MedicationOrder", + "id": "00347e04-8907-43f6-a90d-03c8af72cd5c", + "dateWritten": "2004-08-30T15:14:32-07:00", + "status": "active", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "prescriber": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + }, + "encounter": { + "reference": "urn:uuid:58764f28-1efe-463c-b978-747589b6978b" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "2001499", + "display": "Vitamin B 12 5 MG/ML Injectable Solution" + } + ], + "text": "Vitamin B 12 5 MG/ML Injectable Solution" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:f98510c6-f134-4053-b275-03bcea081a8b", + "resource": { + "resourceType": "Claim", + "id": "f98510c6-f134-4053-b275-03bcea081a8b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:00347e04-8907-43f6-a90d-03c8af72cd5c" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:63e4f590-7837-4253-9a4f-f8a6e3694911", + "resource": { + "resourceType": "Claim", + "id": "63e4f590-7837-4253-9a4f-f8a6e3694911", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1", + "resource": { + "resourceType": "Encounter", + "id": "e8294067-7989-4465-8ec9-2e21c1018fd1", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2010-06-21T15:14:32-07:00", + "end": "2010-06-21T15:44:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:34e20d3d-e0ce-40d8-afae-023d328e8d76", + "resource": { + "resourceType": "Observation", + "id": "34e20d3d-e0ce-40d8-afae-023d328e8d76", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f2022249-5b3e-4d6e-b35c-db47533b057d", + "resource": { + "resourceType": "Observation", + "id": "f2022249-5b3e-4d6e-b35c-db47533b057d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2164fb5d-94f5-4e0a-a99a-ef8ee747851f", + "resource": { + "resourceType": "Observation", + "id": "2164fb5d-94f5-4e0a-a99a-ef8ee747851f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 94.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e0ebed81-17a7-4980-88a7-cc4c7a4f14a4", + "resource": { + "resourceType": "Observation", + "id": "e0ebed81-17a7-4980-88a7-cc4c7a4f14a4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 30.140, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e31ccabb-8298-4fc0-8ff0-4f46049a03f7", + "resource": { + "resourceType": "Observation", + "id": "e31ccabb-8298-4fc0-8ff0-4f46049a03f7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:08d0c372-db13-47b5-ac4b-02c08c76c4af", + "resource": { + "resourceType": "Observation", + "id": "08d0c372-db13-47b5-ac4b-02c08c76c4af", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 87, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:608899af-a8f3-41ac-804b-41057e3e87a1", + "resource": { + "resourceType": "Observation", + "id": "608899af-a8f3-41ac-804b-41057e3e87a1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bea5fe32-a356-42f2-9c96-9fc58b2581cd", + "resource": { + "resourceType": "Observation", + "id": "bea5fe32-a356-42f2-9c96-9fc58b2581cd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 80.840, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc6276d2-1abe-4887-bedf-bfb68bf72751", + "resource": { + "resourceType": "Observation", + "id": "dc6276d2-1abe-4887-bedf-bfb68bf72751", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 13.090, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1da7438a-81f3-4be2-b3bc-bdfcd955dd50", + "resource": { + "resourceType": "Observation", + "id": "1da7438a-81f3-4be2-b3bc-bdfcd955dd50", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.3100, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:093db9d0-960f-4d87-9e2f-1ff1e2725aff", + "resource": { + "resourceType": "Observation", + "id": "093db9d0-960f-4d87-9e2f-1ff1e2725aff", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 10.120, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:94d3fb0c-cb19-4804-bd23-a6c935c46b76", + "resource": { + "resourceType": "Observation", + "id": "94d3fb0c-cb19-4804-bd23-a6c935c46b76", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 137.54, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d74d8e7-2274-45f1-a083-c6c94d7b35aa", + "resource": { + "resourceType": "Observation", + "id": "9d74d8e7-2274-45f1-a083-c6c94d7b35aa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 4.7300, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:979ba582-8d53-4717-a2be-7bf9139a18a6", + "resource": { + "resourceType": "Observation", + "id": "979ba582-8d53-4717-a2be-7bf9139a18a6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 106.38, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:88c292bf-1c48-4971-8407-4cec469890f1", + "resource": { + "resourceType": "Observation", + "id": "88c292bf-1c48-4971-8407-4cec469890f1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 25.060, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d71f372b-60d3-4420-9dad-8b05c87d1ab1", + "resource": { + "resourceType": "Observation", + "id": "d71f372b-60d3-4420-9dad-8b05c87d1ab1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 6.5061, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dbde21c8-aa29-4126-9f5b-9ad4a7a598f0", + "resource": { + "resourceType": "Observation", + "id": "dbde21c8-aa29-4126-9f5b-9ad4a7a598f0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.0094, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6cbb1ea8-fb5e-4fef-9bd1-e7b00704df6d", + "resource": { + "resourceType": "Observation", + "id": "6cbb1ea8-fb5e-4fef-9bd1-e7b00704df6d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 14.763, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b4db91da-093d-49f7-b2bf-48aeafacf6d6", + "resource": { + "resourceType": "Observation", + "id": "b4db91da-093d-49f7-b2bf-48aeafacf6d6", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 47.219, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b79e5d17-aa3b-4832-8f32-fc3172671f3f", + "resource": { + "resourceType": "Observation", + "id": "b79e5d17-aa3b-4832-8f32-fc3172671f3f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 93.731, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:60cff5b5-e0e4-4b99-a321-e013d2dd6cef", + "resource": { + "resourceType": "Observation", + "id": "60cff5b5-e0e4-4b99-a321-e013d2dd6cef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 32.546, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:48dcb9e6-1f7a-4c66-b44e-3d1644724c3b", + "resource": { + "resourceType": "Observation", + "id": "48dcb9e6-1f7a-4c66-b44e-3d1644724c3b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 34.765, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e62e5b25-2399-442e-a717-0356e5e72f29", + "resource": { + "resourceType": "Observation", + "id": "e62e5b25-2399-442e-a717-0356e5e72f29", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 40.522, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5448a7ed-3736-4df0-8d9e-0341ca12029a", + "resource": { + "resourceType": "Observation", + "id": "5448a7ed-3736-4df0-8d9e-0341ca12029a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 261.03, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dac3b08f-1084-4642-9d86-27ab5b70135c", + "resource": { + "resourceType": "Observation", + "id": "dac3b08f-1084-4642-9d86-27ab5b70135c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 376.63, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6e51f20-9c12-44d6-b6c0-f10618710944", + "resource": { + "resourceType": "Observation", + "id": "c6e51f20-9c12-44d6-b6c0-f10618710944", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 11.045, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9fbe90e8-5196-4bd5-b4c2-2a1b5a38f44b", + "resource": { + "resourceType": "Observation", + "id": "9fbe90e8-5196-4bd5-b4c2-2a1b5a38f44b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67bd38b6-626d-45c8-8fe6-ab5312bf6f01", + "resource": { + "resourceType": "Observation", + "id": "67bd38b6-626d-45c8-8fe6-ab5312bf6f01", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.8400, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e888fbd3-265e-46de-ae6f-9d339c767486", + "resource": { + "resourceType": "Procedure", + "id": "e888fbd3-265e-46de-ae6f-9d339c767486", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2010-06-21T15:14:32-07:00", + "end": "2010-06-21T15:29:32-07:00" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e324eaa2-cb14-424b-a013-102df2c1b97e", + "resource": { + "resourceType": "Immunization", + "id": "e324eaa2-cb14-424b-a013-102df2c1b97e", + "status": "completed", + "date": "2010-06-21T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:89d4cc47-69ac-4e1f-bdc1-ff6b4a50d7c4", + "resource": { + "resourceType": "Immunization", + "id": "89d4cc47-69ac-4e1f-bdc1-ff6b4a50d7c4", + "status": "completed", + "date": "2010-06-21T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + } + ], + "text": "Hep A, adult" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:33b6483a-d39f-46ba-af42-76083f5c77e7", + "resource": { + "resourceType": "DiagnosticReport", + "id": "33b6483a-d39f-46ba-af42-76083f5c77e7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:88c292bf-1c48-4971-8407-4cec469890f1", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:8dc0a984-da94-4485-86e1-286bc6afe1df", + "resource": { + "resourceType": "DiagnosticReport", + "id": "8dc0a984-da94-4485-86e1-286bc6afe1df", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" + }, + "effectiveDateTime": "2010-06-21T15:14:32-07:00", + "issued": "2010-06-21T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:c6e51f20-9c12-44d6-b6c0-f10618710944", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0014b041-5051-42b0-b302-557879cc3bbb", + "resource": { + "resourceType": "Claim", + "id": "0014b041-5051-42b0-b302-557879cc3bbb", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 651.64, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805", + "resource": { + "resourceType": "Encounter", + "id": "f124f66b-fa74-4122-8901-941e80326805", + "status": "finished", + "class": "emergency", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2010-12-10T14:14:32-08:00", + "end": "2010-12-10T16:20:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961", + "resource": { + "resourceType": "Condition", + "id": "e13c18a8-21c1-48ba-bb7a-a6b1c2cde961", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + }, + "dateRecorded": "2010-12-10", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "33737001", + "display": "Fracture of rib" + } + ], + "text": "Fracture of rib" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2010-12-10T14:14:32-08:00", + "abatementDateTime": "2011-03-10T14:14:32-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:eb7910bc-c08d-4307-be88-ace1f360436d", + "resource": { + "resourceType": "Procedure", + "id": "eb7910bc-c08d-4307-be88-ace1f360436d", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "399208008", + "display": "Chest X-ray" + } + ], + "text": "Chest X-ray" + }, + "performedPeriod": { + "start": "2010-12-10T14:14:32-08:00", + "end": "2010-12-10T14:44:32-08:00" + }, + "encounter": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f17528ad-4251-4346-9825-578a3152c169", + "resource": { + "resourceType": "Procedure", + "id": "f17528ad-4251-4346-9825-578a3152c169", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274474001", + "display": "Bone immobilization" + } + ], + "text": "Bone immobilization" + }, + "reasonReference": { + "reference": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961" + }, + "performedPeriod": { + "start": "2010-12-10T14:14:32-08:00", + "end": "2010-12-10T14:50:32-08:00" + }, + "encounter": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f5db4141-0846-48ef-8e17-6408645a4b91", + "resource": { + "resourceType": "MedicationOrder", + "id": "f5db4141-0846-48ef-8e17-6408645a4b91", + "dateWritten": "2010-12-10T14:14:32-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "prescriber": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + }, + "encounter": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "857005", + "display": "Acetaminophen 325 MG / HYDROcodone Bitartrate 7.5 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG / HYDROcodone Bitartrate 7.5 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:bf0d237a-0418-4305-8569-c6d3f05d565e", + "resource": { + "resourceType": "Claim", + "id": "bf0d237a-0418-4305-8569-c6d3f05d565e", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:f5db4141-0846-48ef-8e17-6408645a4b91" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:350364dd-9d41-478a-9370-daaa70f853ab", + "resource": { + "resourceType": "MedicationOrder", + "id": "350364dd-9d41-478a-9370-daaa70f853ab", + "dateWritten": "2010-12-10T14:14:32-08:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "prescriber": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + }, + "encounter": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "310965", + "display": "Ibuprofen 200 MG Oral Tablet" + } + ], + "text": "Ibuprofen 200 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:77bc50cc-f4cb-4c94-a483-ba156759dc46", + "resource": { + "resourceType": "Claim", + "id": "77bc50cc-f4cb-4c94-a483-ba156759dc46", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:350364dd-9d41-478a-9370-daaa70f853ab" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0fc90883-cda1-41bc-9c0a-4d10e835c8bd", + "resource": { + "resourceType": "CarePlan", + "id": "0fc90883-cda1-41bc-9c0a-4d10e835c8bd", + "text": { + "status": "generated", + "div": "
    Fracture care
    " + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "context": { + "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" + }, + "period": { + "start": "2010-12-10T14:14:32-08:00", + "end": "2011-03-10T14:14:32-08:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385691007", + "display": "Fracture care" + } + ], + "text": "Fracture care" + } + ], + "addresses": [ + { + "reference": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "183051005", + "display": "Recommendation to rest" + } + ], + "text": "Recommendation to rest" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "408580007", + "display": "Physical activity target light exercise" + } + ], + "text": "Physical activity target light exercise" + }, + "status": "completed", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:ac92faef-c98a-4b58-8704-092977a9ae3f", + "resource": { + "resourceType": "ImagingStudy", + "id": "ac92faef-c98a-4b58-8704-092977a9ae3f", + "started": "2010-12-10T14:14:32-08:00", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "uid": "urn:oid:1.2.840.99999999.64685352.1586309774039", + "numberOfSeries": 1, + "numberOfInstances": 1, + "series": [ + { + "number": 1, + "modality": { + "system": "http://dicom.nema.org/resources/ontology/DCM", + "code": "DX", + "display": "Digital Radiography" + }, + "uid": "urn:oid:1.2.840.99999999.1.82492111.1586309774039", + "numberOfInstances": 1, + "availability": "UNAVAILABLE", + "bodySite": { + "system": "http://snomed.info/sct", + "code": "51185008", + "display": "Chest" + }, + "started": "2010-12-10T14:14:32-08:00", + "instance": [ + { + "number": 1, + "uid": "urn:oid:1.2.840.99999999.1.1.11502719.1586309774039", + "sopClass": "urn:oid:1.2.840.10008.5.1.4.1.1.1.1", + "title": "Image of chest" + } + ] + } + ] + }, + "request": { + "method": "POST", + "url": "ImagingStudy" + } + }, + { + "fullUrl": "urn:uuid:527c7821-4dab-4985-b35c-e7b39a578026", + "resource": { + "resourceType": "Claim", + "id": "527c7821-4dab-4985-b35c-e7b39a578026", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "33737001", + "display": "Fracture of rib" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "399208008", + "display": "Chest X-ray" + }, + "net": { + "value": 11005.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "274474001", + "display": "Bone immobilization" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d102fa74-8e90-4f2c-8611-11c823e3ff58", + "resource": { + "resourceType": "Encounter", + "id": "d102fa74-8e90-4f2c-8611-11c823e3ff58", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2011-03-10T14:14:32-08:00", + "end": "2011-03-10T14:29:32-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "33737001", + "display": "Fracture of rib" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6c5cc660-a1d5-4b7c-b547-afa7c12129e7", + "resource": { + "resourceType": "Claim", + "id": "6c5cc660-a1d5-4b7c-b547-afa7c12129e7", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2", + "resource": { + "resourceType": "Encounter", + "id": "dabf9d6a-0972-4d3e-be29-957f348feec2", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2012-04-19T15:14:32-07:00", + "end": "2012-04-19T15:49:32-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0", + "resource": { + "resourceType": "Condition", + "id": "775e4bf0-7f52-4f37-a03f-e44b9ff11ef0", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" + }, + "dateRecorded": "2012-04-19", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2012-04-19T15:14:32-07:00", + "abatementDateTime": "2012-04-26T15:14:32-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:e3afa29d-f93b-41f7-a0d5-a6712fb519c1", + "resource": { + "resourceType": "Procedure", + "id": "e3afa29d-f93b-41f7-a0d5-a6712fb519c1", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "reasonReference": { + "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" + }, + "performedPeriod": { + "start": "2012-04-19T15:14:32-07:00", + "end": "2012-04-19T15:34:32-07:00" + }, + "encounter": { + "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:43ae4b94-925f-4919-aefc-b0f990a1bc5a", + "resource": { + "resourceType": "MedicationOrder", + "id": "43ae4b94-925f-4919-aefc-b0f990a1bc5a", + "dateWritten": "2012-04-19T15:14:32-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "prescriber": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + }, + "encounter": { + "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" + }, + "reasonReference": { + "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:00f48756-4c62-45ab-8ecf-e30ff789f941", + "resource": { + "resourceType": "Claim", + "id": "00f48756-4c62-45ab-8ecf-e30ff789f941", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:43ae4b94-925f-4919-aefc-b0f990a1bc5a" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3470f76e-8524-4284-ac0a-9c2e9dd1c516", + "resource": { + "resourceType": "CarePlan", + "id": "3470f76e-8524-4284-ac0a-9c2e9dd1c516", + "text": { + "status": "generated", + "div": "
    Respiratory therapy
    " + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "context": { + "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" + }, + "period": { + "start": "2012-04-19T15:14:32-07:00", + "end": "2012-04-26T15:14:32-07:00" + }, + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "addresses": [ + { + "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "prohibited": false + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "prohibited": false + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:5db17a29-22a6-4a78-b1b9-49293212e6be", + "resource": { + "resourceType": "Claim", + "id": "5db17a29-22a6-4a78-b1b9-49293212e6be", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393", + "resource": { + "resourceType": "Encounter", + "id": "e1ca29b9-6124-4ed0-a617-f7a8a5855393", + "status": "finished", + "class": "outpatient", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2012-04-30T15:14:32-07:00", + "end": "2012-04-30T15:29:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:451e48cb-04fa-44b3-9588-6b7683e3dfa5", + "resource": { + "resourceType": "Observation", + "id": "451e48cb-04fa-44b3-9588-6b7683e3dfa5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb5dd232-761b-4a23-ab5a-92524b7e8576", + "resource": { + "resourceType": "Observation", + "id": "fb5dd232-761b-4a23-ab5a-92524b7e8576", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c76338a-9661-48b1-9243-7ab0bdc4e518", + "resource": { + "resourceType": "Observation", + "id": "7c76338a-9661-48b1-9243-7ab0bdc4e518", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 88.700, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3b3a87c-9dcf-42fc-96f9-70d834112c63", + "resource": { + "resourceType": "Observation", + "id": "b3b3a87c-9dcf-42fc-96f9-70d834112c63", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 28.400, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99f81bd2-b84b-4cb7-bb70-d602599948ee", + "resource": { + "resourceType": "Observation", + "id": "99f81bd2-b84b-4cb7-bb70-d602599948ee", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9df1cdb-9c4a-4684-b579-5cc5b1975fd8", + "resource": { + "resourceType": "Observation", + "id": "a9df1cdb-9c4a-4684-b579-5cc5b1975fd8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 75, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9bc65f53-46e7-43be-813a-e32de96480cd", + "resource": { + "resourceType": "Observation", + "id": "9bc65f53-46e7-43be-813a-e32de96480cd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:06b17005-c565-41fc-9806-3e5db854618b", + "resource": { + "resourceType": "Observation", + "id": "06b17005-c565-41fc-9806-3e5db854618b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 99.760, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8cecb2af-a412-413c-bf53-f7a0f91d19e9", + "resource": { + "resourceType": "Observation", + "id": "8cecb2af-a412-413c-bf53-f7a0f91d19e9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 16.560, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b9026ff5-71ea-47ac-863e-14e600820083", + "resource": { + "resourceType": "Observation", + "id": "b9026ff5-71ea-47ac-863e-14e600820083", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.2200, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e7965cd0-67d8-4a94-8a3c-f8497c1e2459", + "resource": { + "resourceType": "Observation", + "id": "e7965cd0-67d8-4a94-8a3c-f8497c1e2459", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 9.4300, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40a510fc-ffc9-4867-a351-053697bd708f", + "resource": { + "resourceType": "Observation", + "id": "40a510fc-ffc9-4867-a351-053697bd708f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 143.06, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76bf3529-4893-4151-a821-d2c178889f5e", + "resource": { + "resourceType": "Observation", + "id": "76bf3529-4893-4151-a821-d2c178889f5e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.0100, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d74a1e9-e094-42ee-9522-ced256860d49", + "resource": { + "resourceType": "Observation", + "id": "5d74a1e9-e094-42ee-9522-ced256860d49", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 108.82, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:711a3dc5-ceab-44a4-9cdf-2746d702b541", + "resource": { + "resourceType": "Observation", + "id": "711a3dc5-ceab-44a4-9cdf-2746d702b541", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 27.190, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6c938382-dee3-420b-b7e0-f0ccaf952502", + "resource": { + "resourceType": "Observation", + "id": "6c938382-dee3-420b-b7e0-f0ccaf952502", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 169.57, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:03d82c6b-4aef-47d9-b71c-a40c61cfbd1c", + "resource": { + "resourceType": "Observation", + "id": "03d82c6b-4aef-47d9-b71c-a40c61cfbd1c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 107.81, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3af8274a-8f9f-4c02-8f9e-94c6b664acac", + "resource": { + "resourceType": "Observation", + "id": "3af8274a-8f9f-4c02-8f9e-94c6b664acac", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 75.380, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:880d05f0-b73b-4363-8d6e-66b07362793f", + "resource": { + "resourceType": "Observation", + "id": "880d05f0-b73b-4363-8d6e-66b07362793f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 72.640, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69ae792d-5ed5-440e-88b9-b64b53f7f394", + "resource": { + "resourceType": "Observation", + "id": "69ae792d-5ed5-440e-88b9-b64b53f7f394", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:78853a2c-25bc-4ed8-ac19-163887e55b7e", + "resource": { + "resourceType": "Observation", + "id": "78853a2c-25bc-4ed8-ac19-163887e55b7e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.9600, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9761dc4f-9b1e-48cc-ab4a-077b824d9ff1", + "resource": { + "resourceType": "Immunization", + "id": "9761dc4f-9b1e-48cc-ab4a-077b824d9ff1", + "status": "completed", + "date": "2012-04-30T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:feb9d718-6be7-4c32-8e4c-70d69e85a3e5", + "resource": { + "resourceType": "Immunization", + "id": "feb9d718-6be7-4c32-8e4c-70d69e85a3e5", + "status": "completed", + "date": "2012-04-30T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:febb4be4-d5c5-4e72-a1cc-6b913167b2cd", + "resource": { + "resourceType": "Immunization", + "id": "febb4be4-d5c5-4e72-a1cc-6b913167b2cd", + "status": "completed", + "date": "2012-04-30T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + } + ], + "text": "Hep A, adult" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b7e0cf76-da7a-43dd-be2a-16d7645e1449", + "resource": { + "resourceType": "DiagnosticReport", + "id": "b7e0cf76-da7a-43dd-be2a-16d7645e1449", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "result": [ + { + "reference": "urn:uuid:711a3dc5-ceab-44a4-9cdf-2746d702b541", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f2aa89e6-b0b7-427c-b408-5356683187f4", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f2aa89e6-b0b7-427c-b408-5356683187f4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" + }, + "effectiveDateTime": "2012-04-30T15:14:32-07:00", + "issued": "2012-04-30T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "result": [ + { + "reference": "urn:uuid:880d05f0-b73b-4363-8d6e-66b07362793f", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b3f5fb10-ee9c-4d17-9ad5-ed99460dfcdc", + "resource": { + "resourceType": "Claim", + "id": "b3f5fb10-ee9c-4d17-9ad5-ed99460dfcdc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "52", + "display": "Hep A, adult" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4", + "resource": { + "resourceType": "Encounter", + "id": "873efd7e-4b2d-4ddc-901e-1298549575e4", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2012-06-25T15:14:32-07:00", + "end": "2012-06-25T15:44:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:76acaaeb-feab-45f8-90a5-7cd16e0d0bac", + "resource": { + "resourceType": "Observation", + "id": "76acaaeb-feab-45f8-90a5-7cd16e0d0bac", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e45235d0-d70c-4765-a57a-2d9f7ac15600", + "resource": { + "resourceType": "Observation", + "id": "e45235d0-d70c-4765-a57a-2d9f7ac15600", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:210ae20d-717c-4119-aa79-21e96dd8f549", + "resource": { + "resourceType": "Observation", + "id": "210ae20d-717c-4119-aa79-21e96dd8f549", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 87.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:64b06f8a-773c-4975-a03e-c0e68bdb0797", + "resource": { + "resourceType": "Observation", + "id": "64b06f8a-773c-4975-a03e-c0e68bdb0797", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 28.020, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9eaee75-bb6f-4062-ba6a-29a568e5f558", + "resource": { + "resourceType": "Observation", + "id": "c9eaee75-bb6f-4062-ba6a-29a568e5f558", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 134, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0ab866e-2211-4319-9b7a-9000a7c03282", + "resource": { + "resourceType": "Observation", + "id": "c0ab866e-2211-4319-9b7a-9000a7c03282", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebfe0f30-409d-46f2-b9a2-8b4c6a203a0a", + "resource": { + "resourceType": "Observation", + "id": "ebfe0f30-409d-46f2-b9a2-8b4c6a203a0a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:818d6b41-1c1f-4660-8650-b11a26117115", + "resource": { + "resourceType": "Observation", + "id": "818d6b41-1c1f-4660-8650-b11a26117115", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 66.900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:403d90c0-7d42-4a4e-a6ff-19a45b1bcb1f", + "resource": { + "resourceType": "Observation", + "id": "403d90c0-7d42-4a4e-a6ff-19a45b1bcb1f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 16, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e0a448bd-7986-4c26-8290-43b57c201eef", + "resource": { + "resourceType": "Observation", + "id": "e0a448bd-7986-4c26-8290-43b57c201eef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.1900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31d32f3f-7b1b-4a05-8923-d977e1556af1", + "resource": { + "resourceType": "Observation", + "id": "31d32f3f-7b1b-4a05-8923-d977e1556af1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 10.030, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:739ea4cb-b15e-4461-9291-d037db12946d", + "resource": { + "resourceType": "Observation", + "id": "739ea4cb-b15e-4461-9291-d037db12946d", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 136.54, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a4144224-ca59-490f-b099-5baf1a192e10", + "resource": { + "resourceType": "Observation", + "id": "a4144224-ca59-490f-b099-5baf1a192e10", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 4.4000, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ebcf742-da66-4254-92e6-14f8b70cba67", + "resource": { + "resourceType": "Observation", + "id": "1ebcf742-da66-4254-92e6-14f8b70cba67", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 107.30, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31124929-25dd-4d51-bc45-d22b8553c136", + "resource": { + "resourceType": "Observation", + "id": "31124929-25dd-4d51-bc45-d22b8553c136", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 23.410, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca3ebab8-5c95-4857-b7e2-f337a07cc2e7", + "resource": { + "resourceType": "Observation", + "id": "ca3ebab8-5c95-4857-b7e2-f337a07cc2e7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35d245ee-d6af-4509-a2c4-938f146e2cd3", + "resource": { + "resourceType": "Observation", + "id": "35d245ee-d6af-4509-a2c4-938f146e2cd3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "valueQuantity": { + "value": 6.1000, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:360e5870-25bf-4342-ad22-49cd8e0847c2", + "resource": { + "resourceType": "Procedure", + "id": "360e5870-25bf-4342-ad22-49cd8e0847c2", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2012-06-25T15:14:32-07:00", + "end": "2012-06-25T15:29:32-07:00" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f82d5b51-cadd-4ac0-be01-57b30a8c6b0f", + "resource": { + "resourceType": "Immunization", + "id": "f82d5b51-cadd-4ac0-be01-57b30a8c6b0f", + "status": "completed", + "date": "2012-06-25T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c2c01e9c-0151-4dd3-a500-1ded5efd49a5", + "resource": { + "resourceType": "DiagnosticReport", + "id": "c2c01e9c-0151-4dd3-a500-1ded5efd49a5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" + }, + "effectiveDateTime": "2012-06-25T15:14:32-07:00", + "issued": "2012-06-25T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:31124929-25dd-4d51-bc45-d22b8553c136", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b3477957-59a2-4a37-b258-ca2fe6200eba", + "resource": { + "resourceType": "Claim", + "id": "b3477957-59a2-4a37-b258-ca2fe6200eba", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 635.53, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8978465e-2136-41bd-890e-672c8d98f978", + "resource": { + "resourceType": "Encounter", + "id": "8978465e-2136-41bd-890e-672c8d98f978", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2013-02-23T14:14:32-08:00", + "end": "2013-02-23T14:29:32-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1826a772-93d8-4ec1-98c3-1da624bbc0df", + "resource": { + "resourceType": "Condition", + "id": "1826a772-93d8-4ec1-98c3-1da624bbc0df", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:8978465e-2136-41bd-890e-672c8d98f978" + }, + "dateRecorded": "2013-02-23", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2013-02-23T14:14:32-08:00", + "abatementDateTime": "2013-03-09T14:14:32-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ce88550b-de77-4e2d-a157-8698243a5dfc", + "resource": { + "resourceType": "Claim", + "id": "ce88550b-de77-4e2d-a157-8698243a5dfc", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641", + "resource": { + "resourceType": "Encounter", + "id": "eed22565-60eb-4729-b7f5-896802b09641", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2014-06-30T15:14:32-07:00", + "end": "2014-06-30T15:44:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4fd1696c-fa99-43b8-ad3b-3abbf234a614", + "resource": { + "resourceType": "Observation", + "id": "4fd1696c-fa99-43b8-ad3b-3abbf234a614", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d42f456-ecce-4e9e-ad34-6d20db8ab120", + "resource": { + "resourceType": "Observation", + "id": "5d42f456-ecce-4e9e-ad34-6d20db8ab120", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e19bf84d-0689-49e1-a5d8-ba51d76aa8d1", + "resource": { + "resourceType": "Observation", + "id": "e19bf84d-0689-49e1-a5d8-ba51d76aa8d1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 90, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:afcf176a-69ed-4256-9bca-1ce65590731e", + "resource": { + "resourceType": "Observation", + "id": "afcf176a-69ed-4256-9bca-1ce65590731e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 28.840, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd5f84ee-3b40-4932-a91b-6b13b2e24bb1", + "resource": { + "resourceType": "Observation", + "id": "dd5f84ee-3b40-4932-a91b-6b13b2e24bb1", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 125, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e36207d5-1342-485b-84db-d0cecd97910a", + "resource": { + "resourceType": "Observation", + "id": "e36207d5-1342-485b-84db-d0cecd97910a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 100, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9acd6700-cf70-498b-9bae-7465c87dd32f", + "resource": { + "resourceType": "Observation", + "id": "9acd6700-cf70-498b-9bae-7465c87dd32f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:455fdc2c-e12f-4049-b627-1040fa5b3aab", + "resource": { + "resourceType": "Observation", + "id": "455fdc2c-e12f-4049-b627-1040fa5b3aab", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 93.390, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3044abe1-dde8-4b49-bc17-c810c9835341", + "resource": { + "resourceType": "Observation", + "id": "3044abe1-dde8-4b49-bc17-c810c9835341", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 8.6600, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5b84e319-3b02-4297-b414-ee65dcb906a5", + "resource": { + "resourceType": "Observation", + "id": "5b84e319-3b02-4297-b414-ee65dcb906a5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.2000, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c27010f-ad80-451d-a0de-ffacf4ceb5fd", + "resource": { + "resourceType": "Observation", + "id": "3c27010f-ad80-451d-a0de-ffacf4ceb5fd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 9.4500, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9cdd64a8-0ceb-49ae-8bfd-5036703c2b67", + "resource": { + "resourceType": "Observation", + "id": "9cdd64a8-0ceb-49ae-8bfd-5036703c2b67", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 143.41, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c177516-908e-449e-9754-0df337a5f275", + "resource": { + "resourceType": "Observation", + "id": "2c177516-908e-449e-9754-0df337a5f275", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 3.7900, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5dd27b3d-397e-455e-beab-956ad546db04", + "resource": { + "resourceType": "Observation", + "id": "5dd27b3d-397e-455e-beab-956ad546db04", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 108.61, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ab0dd46-8cd8-4290-8e96-76d6f26036a4", + "resource": { + "resourceType": "Observation", + "id": "2ab0dd46-8cd8-4290-8e96-76d6f26036a4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 22.180, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b5da3c37-0527-415c-aaf5-894e419a6f26", + "resource": { + "resourceType": "Observation", + "id": "b5da3c37-0527-415c-aaf5-894e419a6f26", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b33a4c9-50fe-4599-ab1a-b333f07810f7", + "resource": { + "resourceType": "Observation", + "id": "3b33a4c9-50fe-4599-ab1a-b333f07810f7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.9500, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83fdf235-73f5-4484-acbb-61c4a800055e", + "resource": { + "resourceType": "Procedure", + "id": "83fdf235-73f5-4484-acbb-61c4a800055e", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "performedPeriod": { + "start": "2014-06-30T15:14:32-07:00", + "end": "2014-06-30T15:29:32-07:00" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f9c1747b-4233-4254-b600-4936fc774839", + "resource": { + "resourceType": "Immunization", + "id": "f9c1747b-4233-4254-b600-4936fc774839", + "status": "completed", + "date": "2014-06-30T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:61355308-bb5e-4491-8b5b-a872c0ec30fa", + "resource": { + "resourceType": "DiagnosticReport", + "id": "61355308-bb5e-4491-8b5b-a872c0ec30fa", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" + }, + "effectiveDateTime": "2014-06-30T15:14:32-07:00", + "issued": "2014-06-30T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:2ab0dd46-8cd8-4290-8e96-76d6f26036a4", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:fdd528a6-5d63-4798-8107-9b57be4c82b4", + "resource": { + "resourceType": "Claim", + "id": "fdd528a6-5d63-4798-8107-9b57be4c82b4", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 490.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481", + "resource": { + "resourceType": "Encounter", + "id": "7482b173-fb3c-4239-8d43-082d6ed7c481", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2016-07-04T15:14:32-07:00", + "end": "2016-07-04T15:29:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8bc0126f-d281-43c0-ab46-eb5319dfa430", + "resource": { + "resourceType": "Observation", + "id": "8bc0126f-d281-43c0-ab46-eb5319dfa430", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:433113f8-9e45-444b-b6af-9ce64ab732e9", + "resource": { + "resourceType": "Observation", + "id": "433113f8-9e45-444b-b6af-9ce64ab732e9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ccf232f-da8a-472f-9b26-ed1cc7a991f0", + "resource": { + "resourceType": "Observation", + "id": "1ccf232f-da8a-472f-9b26-ed1cc7a991f0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 93.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7cfb768-6973-4722-bae6-51b6b4bcd4f4", + "resource": { + "resourceType": "Observation", + "id": "a7cfb768-6973-4722-bae6-51b6b4bcd4f4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 30.070, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:44f6ae8b-a2b1-4e9f-aaaf-8991a7c51451", + "resource": { + "resourceType": "Observation", + "id": "44f6ae8b-a2b1-4e9f-aaaf-8991a7c51451", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 125, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:24f44567-c7df-463c-8288-c0c2ed6efa88", + "resource": { + "resourceType": "Observation", + "id": "24f44567-c7df-463c-8288-c0c2ed6efa88", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 76, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9d4a2ab-0bf8-457d-89aa-9cf0b6716ef0", + "resource": { + "resourceType": "Observation", + "id": "a9d4a2ab-0bf8-457d-89aa-9cf0b6716ef0", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2c4624b-9709-464d-8efd-c837d247abfd", + "resource": { + "resourceType": "Observation", + "id": "a2c4624b-9709-464d-8efd-c837d247abfd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 65.420, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bdb853cb-31bf-44e5-8f2d-543f3afb7c21", + "resource": { + "resourceType": "Observation", + "id": "bdb853cb-31bf-44e5-8f2d-543f3afb7c21", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 11.310, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:009b42e2-42d3-4ae8-a3ba-1983f7841d26", + "resource": { + "resourceType": "Observation", + "id": "009b42e2-42d3-4ae8-a3ba-1983f7841d26", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.2300, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f91abd1d-f0a8-4fda-ac43-eecd7800efd4", + "resource": { + "resourceType": "Observation", + "id": "f91abd1d-f0a8-4fda-ac43-eecd7800efd4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 9.1500, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e39ea985-1c8d-4aca-b4ec-fbb58b2ef101", + "resource": { + "resourceType": "Observation", + "id": "e39ea985-1c8d-4aca-b4ec-fbb58b2ef101", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 142.53, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5fcdae25-f68b-46af-b8fb-851f673386e9", + "resource": { + "resourceType": "Observation", + "id": "5fcdae25-f68b-46af-b8fb-851f673386e9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 3.9800, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c8b2cbb-db18-4ec7-9800-cfa7a5973274", + "resource": { + "resourceType": "Observation", + "id": "2c8b2cbb-db18-4ec7-9800-cfa7a5973274", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 106.89, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", + "resource": { + "resourceType": "Observation", + "id": "2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 20.660, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1718324-d11b-4aee-aa35-5c40c4921350", + "resource": { + "resourceType": "Observation", + "id": "d1718324-d11b-4aee-aa35-5c40c4921350", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 174.91, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5aacdec5-80e3-4563-86ef-fefaaa6ba9fd", + "resource": { + "resourceType": "Observation", + "id": "5aacdec5-80e3-4563-86ef-fefaaa6ba9fd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 101.86, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c1acfc1e-e102-4d2b-884d-8fd6c51b6e85", + "resource": { + "resourceType": "Observation", + "id": "c1acfc1e-e102-4d2b-884d-8fd6c51b6e85", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 78.310, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84bec837-ecf8-471d-a1f3-b51fe28263c9", + "resource": { + "resourceType": "Observation", + "id": "84bec837-ecf8-471d-a1f3-b51fe28263c9", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 76.220, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e76e0dbf-8992-4794-a692-628902a9530a", + "resource": { + "resourceType": "Observation", + "id": "e76e0dbf-8992-4794-a692-628902a9530a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 9.3247, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc9bd4d1-cec0-4f99-8fa0-632c06214642", + "resource": { + "resourceType": "Observation", + "id": "fc9bd4d1-cec0-4f99-8fa0-632c06214642", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.3298, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:219782f8-d876-46eb-86bc-8d5fc2f6a422", + "resource": { + "resourceType": "Observation", + "id": "219782f8-d876-46eb-86bc-8d5fc2f6a422", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 15.912, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f82ebde3-5dd8-4cde-8fcd-119d72c33e83", + "resource": { + "resourceType": "Observation", + "id": "f82ebde3-5dd8-4cde-8fcd-119d72c33e83", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 46.435, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d5e915c-64e7-4e74-ae43-d80bd7110f96", + "resource": { + "resourceType": "Observation", + "id": "7d5e915c-64e7-4e74-ae43-d80bd7110f96", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 83.143, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:13c324d9-3631-465c-adf9-3b48ccf0bc21", + "resource": { + "resourceType": "Observation", + "id": "13c324d9-3631-465c-adf9-3b48ccf0bc21", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 32.845, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68cc0ac4-786c-4da2-9f30-8b976685565a", + "resource": { + "resourceType": "Observation", + "id": "68cc0ac4-786c-4da2-9f30-8b976685565a", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 34.571, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ffb9a032-a068-49cb-b466-00d73f195ca8", + "resource": { + "resourceType": "Observation", + "id": "ffb9a032-a068-49cb-b466-00d73f195ca8", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 43.802, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e04d93f-524c-48b5-b4f7-14fe8e27506f", + "resource": { + "resourceType": "Observation", + "id": "3e04d93f-524c-48b5-b4f7-14fe8e27506f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 381.72, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0530c7db-fa1a-4dee-86bb-5830b0b53edd", + "resource": { + "resourceType": "Observation", + "id": "0530c7db-fa1a-4dee-86bb-5830b0b53edd", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 396.29, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30bf7ea3-6c8d-428a-8cc6-20439d26b69e", + "resource": { + "resourceType": "Observation", + "id": "30bf7ea3-6c8d-428a-8cc6-20439d26b69e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 9.6093, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0fc3a1d9-ad7b-42bf-948f-e88c451135ef", + "resource": { + "resourceType": "Observation", + "id": "0fc3a1d9-ad7b-42bf-948f-e88c451135ef", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2afe1792-3e43-4b67-9cf9-2e47110bb8a5", + "resource": { + "resourceType": "Observation", + "id": "2afe1792-3e43-4b67-9cf9-2e47110bb8a5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "valueQuantity": { + "value": 6.1200, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e96ae61-06f9-4a98-9617-5adca8e69d25", + "resource": { + "resourceType": "Immunization", + "id": "8e96ae61-06f9-4a98-9617-5adca8e69d25", + "status": "completed", + "date": "2016-07-04T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0fb40172-97c2-4e0e-b42a-56d9c05e4be4", + "resource": { + "resourceType": "DiagnosticReport", + "id": "0fb40172-97c2-4e0e-b42a-56d9c05e4be4", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:e5997cee-7e27-477b-b525-4da465317220", + "resource": { + "resourceType": "DiagnosticReport", + "id": "e5997cee-7e27-477b-b525-4da465317220", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:84bec837-ecf8-471d-a1f3-b51fe28263c9", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:3adba72e-d38c-4a9b-be2a-e860e4ad9a4e", + "resource": { + "resourceType": "DiagnosticReport", + "id": "3adba72e-d38c-4a9b-be2a-e860e4ad9a4e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" + }, + "effectiveDateTime": "2016-07-04T15:14:32-07:00", + "issued": "2016-07-04T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:30bf7ea3-6c8d-428a-8cc6-20439d26b69e", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:280cae6e-d63f-41ad-83ec-d858a3c4044d", + "resource": { + "resourceType": "Claim", + "id": "280cae6e-d63f-41ad-83ec-d858a3c4044d", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a", + "resource": { + "resourceType": "Encounter", + "id": "ae251a8e-616a-4ada-9296-217127bb490a", + "status": "finished", + "class": "ambulatory", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + } + } + ], + "period": { + "start": "2016-10-09T15:14:32-07:00", + "end": "2016-10-09T15:44:32-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd", + "resource": { + "resourceType": "Condition", + "id": "1b764010-ac5a-491e-8ec5-16bab53cd5bd", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" + }, + "dateRecorded": "2016-10-09", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + }, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/condition-category", + "code": "diagnosis" + } + ] + }, + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "onsetDateTime": "2016-10-09T15:14:32-07:00", + "abatementDateTime": "2016-10-18T15:14:32-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:3774a010-5e60-45f5-8d76-a6bfda4b715c", + "resource": { + "resourceType": "Observation", + "id": "3774a010-5e60-45f5-8d76-a6bfda4b715c", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" + }, + "effectiveDateTime": "2016-10-09T15:14:32-07:00", + "issued": "2016-10-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 38.005, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cdfb1614-c8df-48a0-be98-75a0f2caf468", + "resource": { + "resourceType": "Procedure", + "id": "cdfb1614-c8df-48a0-be98-75a0f2caf468", + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "reasonReference": { + "reference": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd" + }, + "performedPeriod": { + "start": "2016-10-09T15:14:32-07:00", + "end": "2016-10-09T15:29:32-07:00" + }, + "encounter": { + "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:1f5f0725-6902-4150-a0df-ed7c23553d5d", + "resource": { + "resourceType": "MedicationOrder", + "id": "1f5f0725-6902-4150-a0df-ed7c23553d5d", + "dateWritten": "2016-10-09T15:14:32-07:00", + "status": "stopped", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "prescriber": { + "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" + }, + "encounter": { + "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" + }, + "reasonReference": { + "reference": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd" + }, + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "834102", + "display": "Penicillin V Potassium 500 MG Oral Tablet" + } + ], + "text": "Penicillin V Potassium 500 MG Oral Tablet" + } + }, + "request": { + "method": "POST", + "url": "MedicationOrder" + } + }, + { + "fullUrl": "urn:uuid:187c7568-54cf-456e-a022-3699bd3d7a67", + "resource": { + "resourceType": "Claim", + "id": "187c7568-54cf-456e-a022-3699bd3d7a67", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "prescription": { + "reference": "urn:uuid:1f5f0725-6902-4150-a0df-ed7c23553d5d" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f4fcb9e1-936f-4fec-9a36-0898866f3e6b", + "resource": { + "resourceType": "Claim", + "id": "f4fcb9e1-936f-4fec-9a36-0898866f3e6b", + "type": "institutional", + "organization": { + "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" + }, + "use": "complete", + "diagnosis": [ + { + "sequence": 1, + "diagnosis": { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + }, + { + "sequence": 3, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + }, + "net": { + "value": 2325.12, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f", + "resource": { + "resourceType": "Encounter", + "id": "9029c1c3-3e90-430f-84d3-2710961f1a5f", + "status": "finished", + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" + } + } + ], + "period": { + "start": "2018-07-09T15:14:32-07:00", + "end": "2018-07-09T15:29:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d6208ddf-79f4-4d9b-bac6-96639bc25963", + "resource": { + "resourceType": "Observation", + "id": "d6208ddf-79f4-4d9b-bac6-96639bc25963", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 176.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e92feaa-b6f5-49ef-bbbb-a693a8643268", + "resource": { + "resourceType": "Observation", + "id": "3e92feaa-b6f5-49ef-bbbb-a693a8643268", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a959df95-0bf6-44c7-86f1-0a68bc276de3", + "resource": { + "resourceType": "Observation", + "id": "a959df95-0bf6-44c7-86f1-0a68bc276de3", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 88, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8562e56f-eab7-4621-a02a-763615fb9e89", + "resource": { + "resourceType": "Observation", + "id": "8562e56f-eab7-4621-a02a-763615fb9e89", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 28.190, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:39a96f7d-d019-4f53-9e0d-c4ffc0d6eecf", + "resource": { + "resourceType": "Observation", + "id": "39a96f7d-d019-4f53-9e0d-c4ffc0d6eecf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 72, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a691721a-19b9-44ad-a0bb-951e2757e629", + "resource": { + "resourceType": "Observation", + "id": "a691721a-19b9-44ad-a0bb-951e2757e629", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4d392dee-f593-4de4-aabe-6fdecd5c055f", + "resource": { + "resourceType": "Observation", + "id": "4d392dee-f593-4de4-aabe-6fdecd5c055f", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ], + "text": "vital-signs" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5cd46b09-84f7-4a7b-a0d7-5c1a8b9b1edf", + "resource": { + "resourceType": "Observation", + "id": "5cd46b09-84f7-4a7b-a0d7-5c1a8b9b1edf", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 88.960, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d659949-e96b-41ab-b4bc-7ef6d45c296b", + "resource": { + "resourceType": "Observation", + "id": "0d659949-e96b-41ab-b4bc-7ef6d45c296b", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 19.420, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cec03bed-4729-4d42-8010-467d80929cec", + "resource": { + "resourceType": "Observation", + "id": "cec03bed-4729-4d42-8010-467d80929cec", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 1.1200, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:695c8154-110e-462a-ab0d-f536809ed44e", + "resource": { + "resourceType": "Observation", + "id": "695c8154-110e-462a-ab0d-f536809ed44e", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 8.6900, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25dca154-f59e-4a62-ab3d-5f0b0e8fc915", + "resource": { + "resourceType": "Observation", + "id": "25dca154-f59e-4a62-ab3d-5f0b0e8fc915", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 141.33, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:215a790f-9080-4fb9-b950-35ba6fc36a41", + "resource": { + "resourceType": "Observation", + "id": "215a790f-9080-4fb9-b950-35ba6fc36a41", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.1800, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:17248f47-1630-4e44-9d77-122d8cb37da5", + "resource": { + "resourceType": "Observation", + "id": "17248f47-1630-4e44-9d77-122d8cb37da5", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 109.62, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f4fb0f1c-0600-4912-8cb3-ca4091597c49", + "resource": { + "resourceType": "Observation", + "id": "f4fb0f1c-0600-4912-8cb3-ca4091597c49", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 27.860, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:562fc911-423f-46f4-ae18-c56071403825", + "resource": { + "resourceType": "Observation", + "id": "562fc911-423f-46f4-ae18-c56071403825", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ], + "text": "survey" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:948cc9ea-644e-4430-aa6b-bce7df6983dc", + "resource": { + "resourceType": "Observation", + "id": "948cc9ea-644e-4430-aa6b-bce7df6983dc", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ], + "text": "laboratory" + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "valueQuantity": { + "value": 5.9600, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba7baf02-d772-4d35-9308-cdbe2c9a3190", + "resource": { + "resourceType": "Immunization", + "id": "ba7baf02-d772-4d35-9308-cdbe2c9a3190", + "status": "completed", + "date": "2018-07-09T15:14:32-07:00", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "wasNotGiven": false, + "reported": false, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + } + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:badb5f6a-073f-4ad6-b0c2-4a733cc40c89", + "resource": { + "resourceType": "DiagnosticReport", + "id": "badb5f6a-073f-4ad6-b0c2-4a733cc40c89", + "status": "final", + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", + "code": "LAB" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "encounter": { + "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" + }, + "effectiveDateTime": "2018-07-09T15:14:32-07:00", + "issued": "2018-07-09T15:14:32.480-07:00", + "performer": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "result": [ + { + "reference": "urn:uuid:f4fb0f1c-0600-4912-8cb3-ca4091597c49", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:6c348dfd-8fe1-4f8e-95f7-6e30e73369ee", + "resource": { + "resourceType": "Claim", + "id": "6c348dfd-8fe1-4f8e-95f7-6e30e73369ee", + "type": "institutional", + "organization": { + "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + }, + "use": "complete", + "patient": { + "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" + }, + "item": [ + { + "sequence": 1, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + }, + { + "sequence": 2, + "type": { + "system": "http://hl7.org/fhir/v3/ActCode", + "code": "CSINV", + "display": "clinical service invoice" + }, + "service": { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Claim" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json new file mode 100644 index 000000000000..4b9fdcc72d56 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json @@ -0,0 +1,825 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12", + "resource": { + "resourceType": "Organization", + "id": "49318f80-bd8b-3fc7-a096-ac43088b0c12", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 21 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "49318f80-bd8b-3fc7-a096-ac43088b0c12" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "COOLEY DICKINSON HOSPITAL INC,THE", + "telecom": [ + { + "system": "phone", + "value": "4135822000" + } + ], + "address": [ + { + "line": [ + "30 LOCUST STREET" + ], + "city": "NORTHAMPTON", + "state": "MA", + "postalCode": "01060", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92", + "resource": { + "resourceType": "Organization", + "id": "ecc51621-0af3-3b35-ac3e-8b1e34022e92", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 145 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ecc51621-0af3-3b35-ac3e-8b1e34022e92" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "SAINT ANNE'S HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5086745600" + } + ], + "address": [ + { + "line": [ + "795 MIDDLE STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02721", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3", + "resource": { + "resourceType": "Organization", + "id": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "BOSTON MEDICAL CENTER CORPORATION-", + "telecom": [ + { + "system": "phone", + "value": "6176388000" + } + ], + "address": [ + { + "line": [ + "1 BOSTON MEDICAL CENTER PLACE" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02118", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9", + "resource": { + "resourceType": "Organization", + "id": "4861d01f-019c-3dac-a153-8334e50919f9", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 119 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "4861d01f-019c-3dac-a153-8334e50919f9" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "NORTH SHORE MEDICAL CENTER -", + "telecom": [ + { + "system": "phone", + "value": "9787411215" + } + ], + "address": [ + { + "line": [ + "81 HIGHLAND AVENUE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7", + "resource": { + "resourceType": "Organization", + "id": "f4e7709c-02f6-37ca-aeea-8247d74e88e7", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 113 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f4e7709c-02f6-37ca-aeea-8247d74e88e7" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "WINCHESTER HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "7817299000" + } + ], + "address": [ + { + "line": [ + "41 HIGHLAND AVENUE" + ], + "city": "WINCHESTER", + "state": "MA", + "postalCode": "01890", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c", + "resource": { + "resourceType": "Organization", + "id": "b7175ab4-bde5-3848-891b-579bccb77c7c", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "b7175ab4-bde5-3848-891b-579bccb77c7c" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "TUFTS MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "6176365000" + } + ], + "address": [ + { + "line": [ + "800 WASHINGTON STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02111", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", + "resource": { + "resourceType": "Organization", + "id": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 39 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 32 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 22 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 12 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "PCP15859", + "telecom": [ + { + "system": "phone", + "value": "413-387-2118" + } + ], + "address": [ + { + "line": [ + "92 MAIN ST" + ], + "city": "FLORENCE", + "state": "MA", + "postalCode": "01062-1460", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219", + "resource": { + "resourceType": "Organization", + "id": "245f252c-be87-3017-8a0b-a04448a97219", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 48 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 264 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 19 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 27 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "245f252c-be87-3017-8a0b-a04448a97219" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "PCP45531", + "telecom": [ + { + "system": "phone", + "value": "617-230-9940" + } + ], + "address": [ + { + "line": [ + "661 MASSACHUSETTS AVE" + ], + "city": "ARLINGTON", + "state": "MA", + "postalCode": "02476-5001", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c", + "resource": { + "resourceType": "Organization", + "id": "f9914571-32b8-36d3-a422-4cdce9fbb86c", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 56 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 415 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 45 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 27 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f9914571-32b8-36d3-a422-4cdce9fbb86c" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "INDEPENDENCE EYE ASSOCIATES, PC", + "telecom": [ + { + "system": "phone", + "value": "508-985-6600" + } + ], + "address": [ + { + "line": [ + "365 FAUNCE CORNER RD" + ], + "city": "DARTMOUTH", + "state": "MA", + "postalCode": "02747-6230", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86", + "resource": { + "resourceType": "Organization", + "id": "70d8b93b-cc76-3c8b-8929-5aea213ecc86", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 18 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 11 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 113 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "70d8b93b-cc76-3c8b-8929-5aea213ecc86" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "BROOKLINE DERMATOLOGY ASSOCIATES, PC", + "telecom": [ + { + "system": "phone", + "value": "617-608-1575" + } + ], + "address": [ + { + "line": [ + "1208 B VFW PKWY" + ], + "city": "WEST ROXBURY", + "state": "MA", + "postalCode": "02132-4350", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e", + "resource": { + "resourceType": "Organization", + "id": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 37 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 280 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 13 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 22 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "COMMONWEALTH RADIOLOGY ASSOCIATES, INC", + "telecom": [ + { + "system": "phone", + "value": "978-741-1200" + } + ], + "address": [ + { + "line": [ + "81 HIGHLAND AVE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970-2714", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:a76bda1d-f42d-3046-b8c6-021f3db0a086", + "resource": { + "resourceType": "Organization", + "id": "a76bda1d-f42d-3046-b8c6-021f3db0a086", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "a76bda1d-f42d-3046-b8c6-021f3db0a086" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "WALK IN AND PRIMARY CARE", + "telecom": [ + { + "system": "phone", + "value": "781-894-8200" + } + ], + "address": [ + { + "line": [ + "808 MAIN STREET" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2451", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d", + "resource": { + "resourceType": "Organization", + "id": "f7a6cc79-3075-3124-9f62-6fe562440b1d", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "f7a6cc79-3075-3124-9f62-6fe562440b1d" + } + ], + "type": { + "coding": [ + { + "system": "Healthcare Provider", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + }, + "name": "HAWTHORN MEDICAL URGENT CARE CENTER", + "telecom": [ + { + "system": "phone", + "value": "508-961-0861" + } + ], + "address": [ + { + "line": [ + "237 STATE ROAD" + ], + "city": "NORTH DARTMOUTH", + "state": "MA", + "postalCode": "2747", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json new file mode 100644 index 000000000000..59d8bb2107c0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json @@ -0,0 +1,617 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7", + "resource": { + "resourceType": "Practitioner", + "id": "c1a3f738-c767-30ab-951c-5c60f3c569c7", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 21 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "60" + } + ], + "active": true, + "name": { + "family": [ + "Hermiston71" + ], + "given": [ + "Andrea7" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "30 LOCUST STREET" + ], + "city": "NORTHAMPTON", + "state": "MA", + "postalCode": "01060", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d", + "resource": { + "resourceType": "Practitioner", + "id": "38a05f68-cc3b-33b6-993e-1d2bb2696c3d", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 145 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "100" + } + ], + "active": true, + "name": { + "family": [ + "Padberg411" + ], + "given": [ + "Ligia986" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "795 MIDDLE STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02721", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", + "resource": { + "resourceType": "Practitioner", + "id": "64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "140" + } + ], + "active": true, + "name": { + "family": [ + "Kuvalis369" + ], + "given": [ + "Maricruz991" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "1 BOSTON MEDICAL CENTER PLACE" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02118", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e", + "resource": { + "resourceType": "Practitioner", + "id": "b102f703-9e8c-37a2-8eea-08b67c680a1e", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 119 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "160" + } + ], + "active": true, + "name": { + "family": [ + "Roob72" + ], + "given": [ + "Isobel140" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "81 HIGHLAND AVENUE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203", + "resource": { + "resourceType": "Practitioner", + "id": "a0153fac-7137-30d8-bb5a-0cb7af968203", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 113 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "430" + } + ], + "active": true, + "name": { + "family": [ + "Carroll471" + ], + "given": [ + "Emmitt44" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "41 HIGHLAND AVENUE" + ], + "city": "WINCHESTER", + "state": "MA", + "postalCode": "01890", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044", + "resource": { + "resourceType": "Practitioner", + "id": "002862dc-5ff1-380e-82ad-a3cf9c436044", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "470" + } + ], + "active": true, + "name": { + "family": [ + "Orn563" + ], + "given": [ + "Serina556" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "800 WASHINGTON STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02111", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd", + "resource": { + "resourceType": "Practitioner", + "id": "6d5162ad-3294-398e-9eb4-7021465d4bcd", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 39 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10930" + } + ], + "active": true, + "name": { + "family": [ + "Pfeffer420" + ], + "given": [ + "Darryl392" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "92 MAIN ST" + ], + "city": "FLORENCE", + "state": "MA", + "postalCode": "01062-1460", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240", + "resource": { + "resourceType": "Practitioner", + "id": "8e9a1427-af53-3468-9d67-db3b9191c240", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 48 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "28020" + } + ], + "active": true, + "name": { + "family": [ + "Waters156" + ], + "given": [ + "Damien170" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "661 MASSACHUSETTS AVE" + ], + "city": "ARLINGTON", + "state": "MA", + "postalCode": "02476-5001", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b", + "resource": { + "resourceType": "Practitioner", + "id": "7e727355-643f-3e2b-b89f-f5076660354b", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 56 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "61860" + } + ], + "active": true, + "name": { + "family": [ + "Salazar800" + ], + "given": [ + "Rodrigo242" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "365 FAUNCE CORNER RD" + ], + "city": "DARTMOUTH", + "state": "MA", + "postalCode": "02747-6230", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e", + "resource": { + "resourceType": "Practitioner", + "id": "2d307854-d23a-337c-8215-e37f84fdc26e", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 18 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "66350" + } + ], + "active": true, + "name": { + "family": [ + "Eichmann909" + ], + "given": [ + "Marylou497" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "1208 B VFW PKWY" + ], + "city": "WEST ROXBURY", + "state": "MA", + "postalCode": "02132-4350", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc", + "resource": { + "resourceType": "Practitioner", + "id": "f4f50d8e-b084-3bc2-a787-30f3a29983fc", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 37 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "78430" + } + ], + "active": true, + "name": { + "family": [ + "Keeling57" + ], + "given": [ + "Johana303" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "81 HIGHLAND AVE" + ], + "city": "SALEM", + "state": "MA", + "postalCode": "01970-2714", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:2b592db4-9f8b-3284-bac5-60dc80ccce4c", + "resource": { + "resourceType": "Practitioner", + "id": "2b592db4-9f8b-3284-bac5-60dc80ccce4c", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93270" + } + ], + "active": true, + "name": { + "family": [ + "Padberg411" + ], + "given": [ + "Ligia986" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "808 MAIN STREET" + ], + "city": "WALTHAM", + "state": "MA", + "postalCode": "2451", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a", + "resource": { + "resourceType": "Practitioner", + "id": "d4b02c0c-6cc4-31dd-bc8a-817f786b043a", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "93530" + } + ], + "active": true, + "name": { + "family": [ + "Doyle959" + ], + "given": [ + "Willis868" + ], + "prefix": [ + "Dr." + ] + }, + "address": [ + { + "line": [ + "237 STATE ROAD" + ], + "city": "NORTH DARTMOUTH", + "state": "MA", + "postalCode": "2747", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json new file mode 100644 index 000000000000..3bedbb9b820d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json @@ -0,0 +1,16391 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "resource": { + "resourceType": "Patient", + "id": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -6045139225077477987 Population seed: 1586368870505
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Pura348 Grant908" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", + "valueAddress": { + "city": "Boston", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 1.3903699305216464 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 50.60963006947836 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "SS", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-80-1133" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99981444" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "PPN", + "display": "Passport Number" + } + ], + "text": "Passport Number" + }, + "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", + "value": "X32551053X" + } + ], + "name": [ + { + "use": "official", + "family": "Prohaska837", + "given": [ + "Emerson869" + ], + "prefix": [ + "Mr." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-300-9464", + "use": "home" + } + ], + "gender": "male", + "birthDate": "1968-01-01", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.582871446661336 + }, + { + "url": "longitude", + "valueDecimal": -71.29215663584894 + } + ] + } + ], + "line": [ + "113 Toy Mews Unit 67" + ], + "city": "Billerica", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", + "code": "M", + "display": "M" + } + ], + "text": "M" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "resource": { + "resourceType": "Organization", + "id": "b0e04623-b02c-3f8b-92ea-943fc4db60da", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "b0e04623-b02c-3f8b-92ea-943fc4db60da" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "LOWELL GENERAL HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "9789376000" + } + ], + "address": [ + { + "line": [ + "295 VARNUM AVENUE" + ], + "city": "LOWELL", + "state": "MA", + "postalCode": "01854", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "resource": { + "resourceType": "Practitioner", + "id": "57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999759" + } + ], + "active": true, + "name": [ + { + "family": "Tillman293", + "given": [ + "Franklin857" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Franklin857.Tillman293@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "295 VARNUM AVENUE" + ], + "city": "LOWELL", + "state": "MA", + "postalCode": "01854", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948", + "resource": { + "resourceType": "Encounter", + "id": "a9cda84f-edc5-442b-abb6-45625823f948", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "individual": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + } + ], + "period": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "446096008", + "display": "Perennial allergic rhinitis" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d", + "resource": { + "resourceType": "Condition", + "id": "7d6bd517-ca1c-4418-8d83-db5eb534a11d", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "446096008", + "display": "Perennial allergic rhinitis" + } + ], + "text": "Perennial allergic rhinitis" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" + }, + "onsetDateTime": "1972-01-07T04:21:42-08:00", + "recordedDate": "1972-01-07T04:21:42-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:6e71bd2a-e1e4-428b-ad68-f13dea7024ac", + "resource": { + "resourceType": "MedicationRequest", + "id": "6e71bd2a-e1e4-428b-ad68-f13dea7024ac", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "477045", + "display": "Chlorpheniramine Maleate 2 MG/ML Oral Solution" + } + ], + "text": "Chlorpheniramine Maleate 2 MG/ML Oral Solution" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" + }, + "authoredOn": "1972-01-21T04:21:42-08:00", + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:196aa54f-3941-49dd-a683-c97cce2fbd7f", + "resource": { + "resourceType": "Claim", + "id": "196aa54f-3941-49dd-a683-c97cce2fbd7f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "created": "1972-01-21T04:21:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:6e71bd2a-e1e4-428b-ad68-f13dea7024ac" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" + } + ] + } + ], + "total": { + "value": 6.37, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:683826c9-9958-41bb-8068-db0949431668", + "resource": { + "resourceType": "Claim", + "id": "683826c9-9958-41bb-8068-db0949431668", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "created": "1972-01-21T04:21:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "446096008", + "display": "Perennial allergic rhinitis" + } + ], + "text": "Perennial allergic rhinitis" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:55eaebcc-0bb8-46e5-bb2b-7afac26b4fc9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "55eaebcc-0bb8-46e5-bb2b-7afac26b4fc9", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "performer": [ + { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "683826c9-9958-41bb-8068-db0949431668" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "1972-01-21T04:21:42-08:00", + "end": "1973-01-21T04:21:42-08:00" + }, + "created": "1972-01-21T04:21:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:683826c9-9958-41bb-8068-db0949431668" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "446096008", + "display": "Perennial allergic rhinitis" + } + ], + "text": "Perennial allergic rhinitis" + }, + "servicedPeriod": { + "start": "1972-01-07T04:21:42-08:00", + "end": "1972-01-21T04:21:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b", + "resource": { + "resourceType": "Encounter", + "id": "4ec97cae-451d-4cd4-b818-626a34e6980b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1972-01-27T04:21:42-08:00", + "end": "1972-01-27T04:36:42-08:00" + }, + "individual": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + } + ], + "period": { + "start": "1972-01-27T04:21:42-08:00", + "end": "1972-01-27T04:36:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9b6543de-007f-4d5f-a234-48eb078df0ec", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "9b6543de-007f-4d5f-a234-48eb078df0ec", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/allergyintolerance-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/allergyintolerance-verification", + "code": "confirmed" + } + ] + }, + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "419474003", + "display": "Allergy to mould" + } + ], + "text": "Allergy to mould" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "recordedDate": "1972-01-27T04:21:42-08:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:db3bbae7-a750-465a-9b8e-ff8cdec3738c", + "resource": { + "resourceType": "CareTeam", + "id": "db3bbae7-a750-465a-9b8e-ff8cdec3738c", + "status": "active", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" + }, + "period": { + "start": "1972-01-27T04:21:42-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:61bf55ba-45b7-4a7c-a07f-285c81e20d82", + "resource": { + "resourceType": "CarePlan", + "id": "61bf55ba-45b7-4a7c-a07f-285c81e20d82", + "text": { + "status": "generated", + "div": "
    Care Plan for Self-care interventions (procedure).
    Activities:
    • Self-care interventions (procedure)
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "384758001", + "display": "Self-care interventions (procedure)" + } + ], + "text": "Self-care interventions (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" + }, + "period": { + "start": "1972-01-27T04:21:42-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:db3bbae7-a750-465a-9b8e-ff8cdec3738c" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58332002", + "display": "Allergy education" + } + ], + "text": "Allergy education" + }, + "status": "in-progress", + "location": { + "display": "LOWELL GENERAL HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:e2e13816-1dbc-4e1e-af9b-03a924a10218", + "resource": { + "resourceType": "Claim", + "id": "e2e13816-1dbc-4e1e-af9b-03a924a10218", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "1972-01-27T04:21:42-08:00", + "end": "1972-01-27T04:36:42-08:00" + }, + "created": "1972-01-27T04:36:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e511ccf6-e468-4f52-b87c-43ca688051f2", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e511ccf6-e468-4f52-b87c-43ca688051f2", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "performer": [ + { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e2e13816-1dbc-4e1e-af9b-03a924a10218" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "1972-01-27T04:36:42-08:00", + "end": "1973-01-27T04:36:42-08:00" + }, + "created": "1972-01-27T04:36:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e2e13816-1dbc-4e1e-af9b-03a924a10218" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "1972-01-27T04:21:42-08:00", + "end": "1972-01-27T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "resource": { + "resourceType": "Organization", + "id": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC", + "telecom": [ + { + "system": "phone", + "value": "978-667-8600" + } + ], + "address": [ + { + "line": [ + "2 ANDOVER RD" + ], + "city": "BILLERICA", + "state": "MA", + "postalCode": "01821-1916", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "resource": { + "resourceType": "Practitioner", + "id": "0d5ab589-1d26-346a-b8db-e425d2e139e7", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999969899" + } + ], + "active": true, + "name": [ + { + "family": "Gerlach374", + "given": [ + "Lillia547" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Lillia547.Gerlach374@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "2 ANDOVER RD" + ], + "city": "BILLERICA", + "state": "MA", + "postalCode": "01821-1916", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56", + "resource": { + "resourceType": "Encounter", + "id": "404ae370-02ef-4ee0-b161-5700cfa03b56", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5", + "resource": { + "resourceType": "Condition", + "id": "ec171b19-3821-414f-ac72-c38e48aa26d5", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + }, + "onsetDateTime": "1993-03-08T04:21:42-08:00", + "recordedDate": "1993-03-08T04:21:42-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031", + "resource": { + "resourceType": "Condition", + "id": "a226d513-8226-4d13-9a1c-dc0833466031", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + }, + "onsetDateTime": "1993-03-08T04:21:42-08:00", + "recordedDate": "1993-03-08T04:21:42-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f2e9a131-fa38-46c6-b3c9-fc82ac20901b", + "resource": { + "resourceType": "CareTeam", + "id": "f2e9a131-fa38-46c6-b3c9-fc82ac20901b", + "status": "active", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + }, + "period": { + "start": "1993-03-08T04:21:42-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:061c6d94-1977-4e32-9ce6-25e0e2bbc96d", + "resource": { + "resourceType": "CarePlan", + "id": "061c6d94-1977-4e32-9ce6-25e0e2bbc96d", + "text": { + "status": "generated", + "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698360004", + "display": "Diabetes self management plan" + } + ], + "text": "Diabetes self management plan" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + }, + "period": { + "start": "1993-03-08T04:21:42-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:f2e9a131-fa38-46c6-b3c9-fc82ac20901b" + } + ], + "addresses": [ + { + "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "160670007", + "display": "Diabetic diet" + } + ], + "text": "Diabetic diet" + }, + "status": "in-progress", + "location": { + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "location": { + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:15bca152-8ff3-4eb5-9912-3b9756a18bff", + "resource": { + "resourceType": "Claim", + "id": "15bca152-8ff3-4eb5-9912-3b9756a18bff", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "created": "1993-03-08T04:51:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" + } + }, + { + "sequence": 2, + "diagnosisReference": { + "reference": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + }, + { + "sequence": 3, + "diagnosisSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:50b0c5d1-0fc1-48ca-beda-f1527e677d6b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "50b0c5d1-0fc1-48ca-beda-f1527e677d6b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "15bca152-8ff3-4eb5-9912-3b9756a18bff" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "1993-03-08T04:51:42-08:00", + "end": "1994-03-08T04:51:42-08:00" + }, + "created": "1993-03-08T04:51:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:15bca152-8ff3-4eb5-9912-3b9756a18bff" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + }, + { + "sequence": 2, + "diagnosisReference": { + "reference": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "servicedPeriod": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + }, + { + "sequence": 3, + "diagnosisSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "servicedPeriod": { + "start": "1993-03-08T04:21:42-08:00", + "end": "1993-03-08T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d", + "resource": { + "resourceType": "Encounter", + "id": "c3eba345-bfef-45ac-ae1a-2d83edf5ba6d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-01-16T04:21:42-08:00", + "end": "2012-01-16T04:36:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2012-01-16T04:21:42-08:00", + "end": "2012-01-16T04:36:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:405fba50-1d74-4d41-9708-ef2a71508c84", + "resource": { + "resourceType": "Observation", + "id": "405fba50-1d74-4d41-9708-ef2a71508c84", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f645c75e-0677-4f8b-8e21-8738ae538a50", + "resource": { + "resourceType": "Observation", + "id": "f645c75e-0677-4f8b-8e21-8738ae538a50", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fca828f7-bc07-4501-bc02-764e2777f07a", + "resource": { + "resourceType": "Observation", + "id": "fca828f7-bc07-4501-bc02-764e2777f07a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 91.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:720c5016-5da7-4c9f-837a-296672887bb3", + "resource": { + "resourceType": "Observation", + "id": "720c5016-5da7-4c9f-837a-296672887bb3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 30.04, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25277c04-7d44-4427-8eb6-4530fea701eb", + "resource": { + "resourceType": "Observation", + "id": "25277c04-7d44-4427-8eb6-4530fea701eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e99fe67-41d1-410f-bd66-bb19a0a0c641", + "resource": { + "resourceType": "Observation", + "id": "6e99fe67-41d1-410f-bd66-bb19a0a0c641", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28c85ce0-468a-4ebb-98a0-3bd9e95553e1", + "resource": { + "resourceType": "Observation", + "id": "28c85ce0-468a-4ebb-98a0-3bd9e95553e1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a151e5ae-394a-4c28-97d0-9d917d08ea02", + "resource": { + "resourceType": "Observation", + "id": "a151e5ae-394a-4c28-97d0-9d917d08ea02", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 66.93, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:37156042-e715-464a-803c-27662b1a2ba2", + "resource": { + "resourceType": "Observation", + "id": "37156042-e715-464a-803c-27662b1a2ba2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.01, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68b78785-c162-45c4-8258-349e5164235b", + "resource": { + "resourceType": "Observation", + "id": "68b78785-c162-45c4-8258-349e5164235b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.22, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", + "resource": { + "resourceType": "Observation", + "id": "f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 8.73, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", + "resource": { + "resourceType": "Observation", + "id": "dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 139.66, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7fe9e73d-f0af-4447-82a6-4f748bff33b8", + "resource": { + "resourceType": "Observation", + "id": "7fe9e73d-f0af-4447-82a6-4f748bff33b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.22, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:516f75a5-6ea6-4143-8321-0b55f2ca890b", + "resource": { + "resourceType": "Observation", + "id": "516f75a5-6ea6-4143-8321-0b55f2ca890b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 109.91, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff491da6-c0af-40ea-9240-7a63af4828e0", + "resource": { + "resourceType": "Observation", + "id": "ff491da6-c0af-40ea-9240-7a63af4828e0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 21.89, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6deedc99-ea9b-43a9-bfb4-af8fd748c079", + "resource": { + "resourceType": "Observation", + "id": "6deedc99-ea9b-43a9-bfb4-af8fd748c079", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4631f98a-74fc-4daa-b5ff-b81a76035342", + "resource": { + "resourceType": "Observation", + "id": "4631f98a-74fc-4daa-b5ff-b81a76035342", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "valueQuantity": { + "value": 5.94, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d43e29d7-147f-4e85-aceb-bef5252883aa", + "resource": { + "resourceType": "Immunization", + "id": "d43e29d7-147f-4e85-aceb-bef5252883aa", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "occurrenceDateTime": "2012-01-16T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ff140c3e-a905-47b2-af10-d6e4722c1274", + "resource": { + "resourceType": "DiagnosticReport", + "id": "ff140c3e-a905-47b2-af10-d6e4722c1274", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + }, + "effectiveDateTime": "2012-01-16T04:21:42-08:00", + "issued": "2012-01-16T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:a151e5ae-394a-4c28-97d0-9d917d08ea02", + "display": "Glucose" + }, + { + "reference": "urn:uuid:37156042-e715-464a-803c-27662b1a2ba2", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:68b78785-c162-45c4-8258-349e5164235b", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", + "display": "Calcium" + }, + { + "reference": "urn:uuid:dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", + "display": "Sodium" + }, + { + "reference": "urn:uuid:7fe9e73d-f0af-4447-82a6-4f748bff33b8", + "display": "Potassium" + }, + { + "reference": "urn:uuid:516f75a5-6ea6-4143-8321-0b55f2ca890b", + "display": "Chloride" + }, + { + "reference": "urn:uuid:ff491da6-c0af-40ea-9240-7a63af4828e0", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b62dbf14-d8ea-435a-9c33-9684373aa85a", + "resource": { + "resourceType": "Claim", + "id": "b62dbf14-d8ea-435a-9c33-9684373aa85a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2012-01-16T04:21:42-08:00", + "end": "2012-01-16T04:36:42-08:00" + }, + "created": "2012-01-16T04:36:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d43e29d7-147f-4e85-aceb-bef5252883aa" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:398fea0b-50cd-4d00-b362-287d02aa20db", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "398fea0b-50cd-4d00-b362-287d02aa20db", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b62dbf14-d8ea-435a-9c33-9684373aa85a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2012-01-16T04:36:42-08:00", + "end": "2013-01-16T04:36:42-08:00" + }, + "created": "2012-01-16T04:36:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b62dbf14-d8ea-435a-9c33-9684373aa85a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2012-01-16T04:21:42-08:00", + "end": "2012-01-16T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2012-01-16T04:21:42-08:00", + "end": "2012-01-16T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae", + "resource": { + "resourceType": "Encounter", + "id": "29287adc-a638-4419-bb56-80b654c954ae", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-01-20T04:21:42-08:00", + "end": "2014-01-20T04:36:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2014-01-20T04:21:42-08:00", + "end": "2014-01-20T04:36:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5d32ca1a-944f-4668-920c-7d427bb582c2", + "resource": { + "resourceType": "Observation", + "id": "5d32ca1a-944f-4668-920c-7d427bb582c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40b4a2a6-0371-4ac8-a2aa-98a0736a22b7", + "resource": { + "resourceType": "Observation", + "id": "40b4a2a6-0371-4ac8-a2aa-98a0736a22b7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:297b8a08-5868-4c52-9104-c08d3e3661dd", + "resource": { + "resourceType": "Observation", + "id": "297b8a08-5868-4c52-9104-c08d3e3661dd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 91.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2fffaa51-499a-46c0-9d76-68620f71c829", + "resource": { + "resourceType": "Observation", + "id": "2fffaa51-499a-46c0-9d76-68620f71c829", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 30.04, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e890450b-52b2-4f0c-9ef9-b3e344476770", + "resource": { + "resourceType": "Observation", + "id": "e890450b-52b2-4f0c-9ef9-b3e344476770", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34bf394d-0d3d-480f-9ef4-31e04cc3ea93", + "resource": { + "resourceType": "Observation", + "id": "34bf394d-0d3d-480f-9ef4-31e04cc3ea93", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e1695f4-744c-4c4b-9a9c-04b9fab1625c", + "resource": { + "resourceType": "Observation", + "id": "8e1695f4-744c-4c4b-9a9c-04b9fab1625c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", + "resource": { + "resourceType": "Observation", + "id": "b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 76.05, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0b357ed-4ce0-4ae4-97fb-61c179d777f6", + "resource": { + "resourceType": "Observation", + "id": "b0b357ed-4ce0-4ae4-97fb-61c179d777f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 13.88, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69fcb49e-b46c-45b1-bd17-aa7c5792d929", + "resource": { + "resourceType": "Observation", + "id": "69fcb49e-b46c-45b1-bd17-aa7c5792d929", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.3, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:190af0b9-71e7-4668-b685-772ebfca658f", + "resource": { + "resourceType": "Observation", + "id": "190af0b9-71e7-4668-b685-772ebfca658f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 8.62, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6f8610c-c722-45b8-a15d-5c11d8349fa7", + "resource": { + "resourceType": "Observation", + "id": "c6f8610c-c722-45b8-a15d-5c11d8349fa7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 137.49, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", + "resource": { + "resourceType": "Observation", + "id": "8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.49, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c7fe2b8-5a5d-4545-a76a-44791bb28871", + "resource": { + "resourceType": "Observation", + "id": "3c7fe2b8-5a5d-4545-a76a-44791bb28871", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 107.65, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d86173e0-44ba-4d36-bde6-02329d4d46c4", + "resource": { + "resourceType": "Observation", + "id": "d86173e0-44ba-4d36-bde6-02329d4d46c4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 22.57, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67dfe155-480d-4360-8f8e-46a2524e978b", + "resource": { + "resourceType": "Observation", + "id": "67dfe155-480d-4360-8f8e-46a2524e978b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 166, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:94927635-d2ed-4e7a-82bb-baf45c33bf32", + "resource": { + "resourceType": "Observation", + "id": "94927635-d2ed-4e7a-82bb-baf45c33bf32", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 136.52, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f4068fbe-56be-480f-b319-d953ca7f0204", + "resource": { + "resourceType": "Observation", + "id": "f4068fbe-56be-480f-b319-d953ca7f0204", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 59.83, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd747217-d486-408d-94a1-400e8409eec4", + "resource": { + "resourceType": "Observation", + "id": "cd747217-d486-408d-94a1-400e8409eec4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 78.87, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ab25180-451c-4a04-921f-5f53ab235cbc", + "resource": { + "resourceType": "Observation", + "id": "6ab25180-451c-4a04-921f-5f53ab235cbc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 6.6162, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:57bd55ad-75f5-4b90-84e8-3971e44f18af", + "resource": { + "resourceType": "Observation", + "id": "57bd55ad-75f5-4b90-84e8-3971e44f18af", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.0422, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:768eddaa-7c37-494f-861d-ede7b432cd22", + "resource": { + "resourceType": "Observation", + "id": "768eddaa-7c37-494f-861d-ede7b432cd22", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 12.795, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", + "resource": { + "resourceType": "Observation", + "id": "6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 40.741, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", + "resource": { + "resourceType": "Observation", + "id": "a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 89.664, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9b3be03f-2810-4122-8487-1a5d01db4870", + "resource": { + "resourceType": "Observation", + "id": "9b3be03f-2810-4122-8487-1a5d01db4870", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 30.462, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:373801c4-9966-44ad-8e9a-0c99e9cad240", + "resource": { + "resourceType": "Observation", + "id": "373801c4-9966-44ad-8e9a-0c99e9cad240", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 33.524, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9db75a17-a326-4201-a8fa-e2d4119fe5cf", + "resource": { + "resourceType": "Observation", + "id": "9db75a17-a326-4201-a8fa-e2d4119fe5cf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 45.737, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b47d7b7b-7d7e-4051-b511-34fbf912c8c1", + "resource": { + "resourceType": "Observation", + "id": "b47d7b7b-7d7e-4051-b511-34fbf912c8c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 357.81, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:19f52b10-41c6-4d5d-ad2a-7cad3017b343", + "resource": { + "resourceType": "Observation", + "id": "19f52b10-41c6-4d5d-ad2a-7cad3017b343", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 225.32, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0307fe87-f23c-4ce5-aff4-8c367e309cc5", + "resource": { + "resourceType": "Observation", + "id": "0307fe87-f23c-4ce5-aff4-8c367e309cc5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 12.091, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f4f7c83-d77b-4db9-a7ea-2317fed847ca", + "resource": { + "resourceType": "Observation", + "id": "9f4f7c83-d77b-4db9-a7ea-2317fed847ca", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1413d06e-1574-46fe-a7bb-8f266284493c", + "resource": { + "resourceType": "Observation", + "id": "1413d06e-1574-46fe-a7bb-8f266284493c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "valueQuantity": { + "value": 5.98, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d127a6e1-4568-4cd5-8b1d-0ed1c067173a", + "resource": { + "resourceType": "Immunization", + "id": "d127a6e1-4568-4cd5-8b1d-0ed1c067173a", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "occurrenceDateTime": "2014-01-20T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f66881d6-2adc-4d7c-8307-43a2a606b344", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f66881d6-2adc-4d7c-8307-43a2a606b344", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", + "display": "Glucose" + }, + { + "reference": "urn:uuid:b0b357ed-4ce0-4ae4-97fb-61c179d777f6", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:69fcb49e-b46c-45b1-bd17-aa7c5792d929", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:190af0b9-71e7-4668-b685-772ebfca658f", + "display": "Calcium" + }, + { + "reference": "urn:uuid:c6f8610c-c722-45b8-a15d-5c11d8349fa7", + "display": "Sodium" + }, + { + "reference": "urn:uuid:8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", + "display": "Potassium" + }, + { + "reference": "urn:uuid:3c7fe2b8-5a5d-4545-a76a-44791bb28871", + "display": "Chloride" + }, + { + "reference": "urn:uuid:d86173e0-44ba-4d36-bde6-02329d4d46c4", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:a9e8955a-c4b5-4c24-9f3f-7bcd3c6f1351", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a9e8955a-c4b5-4c24-9f3f-7bcd3c6f1351", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:67dfe155-480d-4360-8f8e-46a2524e978b", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:94927635-d2ed-4e7a-82bb-baf45c33bf32", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:f4068fbe-56be-480f-b319-d953ca7f0204", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:cd747217-d486-408d-94a1-400e8409eec4", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7caa0371-603c-4302-8536-2664d2ee63cf", + "resource": { + "resourceType": "DiagnosticReport", + "id": "7caa0371-603c-4302-8536-2664d2ee63cf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + }, + "effectiveDateTime": "2014-01-20T04:21:42-08:00", + "issued": "2014-01-20T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:6ab25180-451c-4a04-921f-5f53ab235cbc", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:57bd55ad-75f5-4b90-84e8-3971e44f18af", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:768eddaa-7c37-494f-861d-ede7b432cd22", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:9b3be03f-2810-4122-8487-1a5d01db4870", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:373801c4-9966-44ad-8e9a-0c99e9cad240", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:9db75a17-a326-4201-a8fa-e2d4119fe5cf", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:b47d7b7b-7d7e-4051-b511-34fbf912c8c1", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:19f52b10-41c6-4d5d-ad2a-7cad3017b343", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:0307fe87-f23c-4ce5-aff4-8c367e309cc5", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:3600f5ed-8ee3-4e14-9063-2a59f891eaec", + "resource": { + "resourceType": "Claim", + "id": "3600f5ed-8ee3-4e14-9063-2a59f891eaec", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2014-01-20T04:21:42-08:00", + "end": "2014-01-20T04:36:42-08:00" + }, + "created": "2014-01-20T04:36:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d127a6e1-4568-4cd5-8b1d-0ed1c067173a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:36be44ca-d517-4e01-b897-0572acff676d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "36be44ca-d517-4e01-b897-0572acff676d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3600f5ed-8ee3-4e14-9063-2a59f891eaec" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2014-01-20T04:36:42-08:00", + "end": "2015-01-20T04:36:42-08:00" + }, + "created": "2014-01-20T04:36:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3600f5ed-8ee3-4e14-9063-2a59f891eaec" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2014-01-20T04:21:42-08:00", + "end": "2014-01-20T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2014-01-20T04:21:42-08:00", + "end": "2014-01-20T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87", + "resource": { + "resourceType": "Encounter", + "id": "314b78fa-de5b-4908-94b4-0846279d1c87", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6999b531-4296-4c14-ac10-bb8169b1f27c", + "resource": { + "resourceType": "Observation", + "id": "6999b531-4296-4c14-ac10-bb8169b1f27c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1fa124b2-1ede-426c-bb65-2c38576c10f8", + "resource": { + "resourceType": "Observation", + "id": "1fa124b2-1ede-426c-bb65-2c38576c10f8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b59c7ed7-84a4-4876-86cf-b6f0392e892e", + "resource": { + "resourceType": "Observation", + "id": "b59c7ed7-84a4-4876-86cf-b6f0392e892e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 83.6, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f995ff32-dbd4-4974-9ca0-69f073e934e5", + "resource": { + "resourceType": "Observation", + "id": "f995ff32-dbd4-4974-9ca0-69f073e934e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 27.34, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:de37dae5-7754-4fc5-81af-98c5a22d19e3", + "resource": { + "resourceType": "Observation", + "id": "de37dae5-7754-4fc5-81af-98c5a22d19e3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c25cd0da-91e0-4726-8948-80cbdabef496", + "resource": { + "resourceType": "Observation", + "id": "c25cd0da-91e0-4726-8948-80cbdabef496", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 81, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f502042b-2438-40bf-92a8-d14b37370247", + "resource": { + "resourceType": "Observation", + "id": "f502042b-2438-40bf-92a8-d14b37370247", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9572bdc-8661-449c-b863-9ab3c4276c66", + "resource": { + "resourceType": "Observation", + "id": "c9572bdc-8661-449c-b863-9ab3c4276c66", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 66.64, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", + "resource": { + "resourceType": "Observation", + "id": "d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 11.82, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aeaa13aa-b5be-49d8-82cd-61814e0f6760", + "resource": { + "resourceType": "Observation", + "id": "aeaa13aa-b5be-49d8-82cd-61814e0f6760", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.07, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", + "resource": { + "resourceType": "Observation", + "id": "f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ff4331b-a76b-4862-9491-7200d6fb17eb", + "resource": { + "resourceType": "Observation", + "id": "0ff4331b-a76b-4862-9491-7200d6fb17eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 140.55, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5fada0a8-adc2-4abd-9f87-fa575582bd43", + "resource": { + "resourceType": "Observation", + "id": "5fada0a8-adc2-4abd-9f87-fa575582bd43", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.25, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:144ac2fd-269c-4b47-bddb-d0605f0dfcef", + "resource": { + "resourceType": "Observation", + "id": "144ac2fd-269c-4b47-bddb-d0605f0dfcef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 109.69, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:735a868b-f60f-4522-b417-5fe4e12785b3", + "resource": { + "resourceType": "Observation", + "id": "735a868b-f60f-4522-b417-5fe4e12785b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 21.45, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33fb567d-a571-4257-973a-52d756b4c36e", + "resource": { + "resourceType": "Observation", + "id": "33fb567d-a571-4257-973a-52d756b4c36e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4bb9225-ac6c-4df9-b7a0-4d31027e72f3", + "resource": { + "resourceType": "Observation", + "id": "d4bb9225-ac6c-4df9-b7a0-4d31027e72f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "valueQuantity": { + "value": 6.24, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e86956a-2972-4fd4-b3ea-f54290189e67", + "resource": { + "resourceType": "Procedure", + "id": "7e86956a-2972-4fd4-b3ea-f54290189e67", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "performedPeriod": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:36:42-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7d52685e-a9a5-4108-b79b-2195600e79d7", + "resource": { + "resourceType": "Immunization", + "id": "7d52685e-a9a5-4108-b79b-2195600e79d7", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "occurrenceDateTime": "2016-01-25T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1126c7df-b0c2-4fad-94d4-69f24207539e", + "resource": { + "resourceType": "DiagnosticReport", + "id": "1126c7df-b0c2-4fad-94d4-69f24207539e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + }, + "effectiveDateTime": "2016-01-25T04:21:42-08:00", + "issued": "2016-01-25T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:c9572bdc-8661-449c-b863-9ab3c4276c66", + "display": "Glucose" + }, + { + "reference": "urn:uuid:d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:aeaa13aa-b5be-49d8-82cd-61814e0f6760", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", + "display": "Calcium" + }, + { + "reference": "urn:uuid:0ff4331b-a76b-4862-9491-7200d6fb17eb", + "display": "Sodium" + }, + { + "reference": "urn:uuid:5fada0a8-adc2-4abd-9f87-fa575582bd43", + "display": "Potassium" + }, + { + "reference": "urn:uuid:144ac2fd-269c-4b47-bddb-d0605f0dfcef", + "display": "Chloride" + }, + { + "reference": "urn:uuid:735a868b-f60f-4522-b417-5fe4e12785b3", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40", + "resource": { + "resourceType": "Claim", + "id": "c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "created": "2016-01-25T04:51:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7d52685e-a9a5-4108-b79b-2195600e79d7" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7e86956a-2972-4fd4-b3ea-f54290189e67" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 456.86, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3cd92632-7d1b-4f78-9cb2-f85ed0b81966", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3cd92632-7d1b-4f78-9cb2-f85ed0b81966", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2016-01-25T04:51:42-08:00", + "end": "2017-01-25T04:51:42-08:00" + }, + "created": "2016-01-25T04:51:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-25T04:21:42-08:00", + "end": "2016-01-25T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 456.86, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 91.37200000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 365.48800000000006, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 456.86, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 456.86, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 477.90400000000005, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3", + "resource": { + "resourceType": "Encounter", + "id": "a8fd8f65-82a3-4b48-890e-e6a1a69489e3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-11-27T04:21:42-08:00", + "end": "2017-11-27T04:36:42-08:00" + }, + "individual": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + } + ], + "period": { + "start": "2017-11-27T04:21:42-08:00", + "end": "2017-11-27T04:36:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6b537404-85fa-4571-94c7-b2aac60aa56d", + "resource": { + "resourceType": "Immunization", + "id": "6b537404-85fa-4571-94c7-b2aac60aa56d", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" + }, + "occurrenceDateTime": "2017-11-27T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:baeb3485-524a-445e-9507-cb8d7faed25c", + "resource": { + "resourceType": "Claim", + "id": "baeb3485-524a-445e-9507-cb8d7faed25c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2017-11-27T04:21:42-08:00", + "end": "2017-11-27T04:36:42-08:00" + }, + "created": "2017-11-27T04:36:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:6b537404-85fa-4571-94c7-b2aac60aa56d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1f8bbb1d-49e9-4fe8-8340-3dfbdcbbc868", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1f8bbb1d-49e9-4fe8-8340-3dfbdcbbc868", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "performer": [ + { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "baeb3485-524a-445e-9507-cb8d7faed25c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2017-11-27T04:36:42-08:00", + "end": "2018-11-27T04:36:42-08:00" + }, + "created": "2017-11-27T04:36:42-08:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:baeb3485-524a-445e-9507-cb8d7faed25c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "2017-11-27T04:21:42-08:00", + "end": "2017-11-27T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2017-11-27T04:21:42-08:00", + "end": "2017-11-27T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317", + "resource": { + "resourceType": "Encounter", + "id": "1a87fe76-cbaa-49a3-9256-db5e5962d317", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "individual": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + } + ], + "period": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff", + "resource": { + "resourceType": "Condition", + "id": "b19cc824-fc9f-4034-9dbe-3232028985ff", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "onsetDateTime": "2017-11-21T04:21:42-08:00", + "abatementDateTime": "2017-12-01T04:21:42-08:00", + "recordedDate": "2017-11-21T04:21:42-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:1c4ee58b-d323-470a-ae25-739d67dc4b5d", + "resource": { + "resourceType": "Observation", + "id": "1c4ee58b-d323-470a-ae25-739d67dc4b5d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + }, + { + "system": "http://loinc.org", + "code": "8331-1", + "display": "Oral temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-21T04:21:42-08:00", + "issued": "2017-11-21T04:21:42.840-08:00", + "valueQuantity": { + "value": 37.019, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:650b0fe7-f02b-44f0-b504-1043b3e937c4", + "resource": { + "resourceType": "Observation", + "id": "650b0fe7-f02b-44f0-b504-1043b3e937c4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b4396c34-bda6-43f6-8b57-c1ae9d155ea8", + "resource": { + "resourceType": "Observation", + "id": "b4396c34-bda6-43f6-8b57-c1ae9d155ea8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:45af5249-feaf-435e-862d-228ae3377222", + "resource": { + "resourceType": "Observation", + "id": "45af5249-feaf-435e-862d-228ae3377222", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 86.6, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8032b18d-63e8-4694-8b16-e5f5ac8e87ce", + "resource": { + "resourceType": "Observation", + "id": "8032b18d-63e8-4694-8b16-e5f5ac8e87ce", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 28.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4ea6b234-08d0-4125-9816-691bc3a44566", + "resource": { + "resourceType": "Observation", + "id": "4ea6b234-08d0-4125-9816-691bc3a44566", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 128, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49d20692-95c1-4a62-86d7-6f15be0a2472", + "resource": { + "resourceType": "Observation", + "id": "49d20692-95c1-4a62-86d7-6f15be0a2472", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2cecf73f-273c-474e-ab83-48d29e79645c", + "resource": { + "resourceType": "Observation", + "id": "2cecf73f-273c-474e-ab83-48d29e79645c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d996a3cd-7ca4-4667-be64-5cd99dc952b0", + "resource": { + "resourceType": "Observation", + "id": "d996a3cd-7ca4-4667-be64-5cd99dc952b0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 96.52, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:80f09d42-3a48-43dd-bc4f-b5544c8b4df0", + "resource": { + "resourceType": "Observation", + "id": "80f09d42-3a48-43dd-bc4f-b5544c8b4df0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 11.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36854602-1113-4729-83d0-9d32df3c8654", + "resource": { + "resourceType": "Observation", + "id": "36854602-1113-4729-83d0-9d32df3c8654", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", + "resource": { + "resourceType": "Observation", + "id": "9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.1, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4f1e36e-2aef-4388-b243-4a54558d0ff8", + "resource": { + "resourceType": "Observation", + "id": "c4f1e36e-2aef-4388-b243-4a54558d0ff8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 140.35, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", + "resource": { + "resourceType": "Observation", + "id": "10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.77, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dbc9df23-758e-4099-b808-a610666fd6e8", + "resource": { + "resourceType": "Observation", + "id": "dbc9df23-758e-4099-b808-a610666fd6e8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 105.2, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70c0597b-1b9c-4322-9d11-111de2d4f8df", + "resource": { + "resourceType": "Observation", + "id": "70c0597b-1b9c-4322-9d11-111de2d4f8df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 26.82, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cbcfa27d-84c6-479a-8607-7ce01a2174ee", + "resource": { + "resourceType": "Observation", + "id": "cbcfa27d-84c6-479a-8607-7ce01a2174ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 198.15, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", + "resource": { + "resourceType": "Observation", + "id": "ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 131.29, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f6cd5cc8-c32b-45e3-87c9-96a19087809a", + "resource": { + "resourceType": "Observation", + "id": "f6cd5cc8-c32b-45e3-87c9-96a19087809a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 94.57, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83a8c0ee-d80a-4162-86b6-3c371c91e79f", + "resource": { + "resourceType": "Observation", + "id": "83a8c0ee-d80a-4162-86b6-3c371c91e79f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 77.32, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac157b02-6bf9-4240-8250-72108a0f907f", + "resource": { + "resourceType": "Observation", + "id": "ac157b02-6bf9-4240-8250-72108a0f907f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d902f66f-14e3-4f38-bfcb-c777e8934668", + "resource": { + "resourceType": "Observation", + "id": "d902f66f-14e3-4f38-bfcb-c777e8934668", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "valueQuantity": { + "value": 6.08, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7b24c86c-b99f-4227-8555-6a87b3285f87", + "resource": { + "resourceType": "Procedure", + "id": "7b24c86c-b99f-4227-8555-6a87b3285f87", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "performedPeriod": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:36:42-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff", + "display": "Acute viral pharyngitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:82fa7941-2a98-4912-9b68-7107a1612d23", + "resource": { + "resourceType": "DiagnosticReport", + "id": "82fa7941-2a98-4912-9b68-7107a1612d23", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:d996a3cd-7ca4-4667-be64-5cd99dc952b0", + "display": "Glucose" + }, + { + "reference": "urn:uuid:80f09d42-3a48-43dd-bc4f-b5544c8b4df0", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:36854602-1113-4729-83d0-9d32df3c8654", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", + "display": "Calcium" + }, + { + "reference": "urn:uuid:c4f1e36e-2aef-4388-b243-4a54558d0ff8", + "display": "Sodium" + }, + { + "reference": "urn:uuid:10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", + "display": "Potassium" + }, + { + "reference": "urn:uuid:dbc9df23-758e-4099-b808-a610666fd6e8", + "display": "Chloride" + }, + { + "reference": "urn:uuid:70c0597b-1b9c-4322-9d11-111de2d4f8df", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:2fff41a0-1ed4-4a68-bccd-7837648fb420", + "resource": { + "resourceType": "DiagnosticReport", + "id": "2fff41a0-1ed4-4a68-bccd-7837648fb420", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + }, + "effectiveDateTime": "2017-11-27T04:21:42-08:00", + "issued": "2017-11-27T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:cbcfa27d-84c6-479a-8607-7ce01a2174ee", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:f6cd5cc8-c32b-45e3-87c9-96a19087809a", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:83a8c0ee-d80a-4162-86b6-3c371c91e79f", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:3632e83a-3b9b-409c-bf8f-213910fc1856", + "resource": { + "resourceType": "Claim", + "id": "3632e83a-3b9b-409c-bf8f-213910fc1856", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "created": "2017-11-21T04:51:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7b24c86c-b99f-4227-8555-6a87b3285f87" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "net": { + "value": 2515.75, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9e902b1d-5977-4010-8540-f9cddbbbb27a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9e902b1d-5977-4010-8540-f9cddbbbb27a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "performer": [ + { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3632e83a-3b9b-409c-bf8f-213910fc1856" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2017-11-21T04:51:42-08:00", + "end": "2018-11-21T04:51:42-08:00" + }, + "created": "2017-11-21T04:51:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3632e83a-3b9b-409c-bf8f-213910fc1856" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "servicedPeriod": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "servicedPeriod": { + "start": "2017-11-21T04:21:42-08:00", + "end": "2017-11-21T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2515.75, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 503.15000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2012.6000000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2515.75, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2515.75, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 2012.6000000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294", + "resource": { + "resourceType": "Encounter", + "id": "e02df952-5c01-4e8b-8cd1-e5f3e7ae8294", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:737add4e-d92a-4bfc-8282-6689449451d7", + "resource": { + "resourceType": "Observation", + "id": "737add4e-d92a-4bfc-8282-6689449451d7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6659c095-2852-4c10-802a-7c8feac75ec3", + "resource": { + "resourceType": "Observation", + "id": "6659c095-2852-4c10-802a-7c8feac75ec3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f74337f3-7d3b-497d-8020-5dd8ef4fc5a0", + "resource": { + "resourceType": "Observation", + "id": "f74337f3-7d3b-497d-8020-5dd8ef4fc5a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 86.9, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7670f173-806f-40c6-8f27-4e7a35447de8", + "resource": { + "resourceType": "Observation", + "id": "7670f173-806f-40c6-8f27-4e7a35447de8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 28.43, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77f0c372-680e-4dff-8b4f-aef5fa2f528b", + "resource": { + "resourceType": "Observation", + "id": "77f0c372-680e-4dff-8b4f-aef5fa2f528b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 135, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d52b4ba4-0818-4a1a-92c7-0a65190125df", + "resource": { + "resourceType": "Observation", + "id": "d52b4ba4-0818-4a1a-92c7-0a65190125df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 66, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:64c8ea7d-4177-4a78-8066-a5ad2fb01aea", + "resource": { + "resourceType": "Observation", + "id": "64c8ea7d-4177-4a78-8066-a5ad2fb01aea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8572f071-3410-4162-b2d4-4e549320795e", + "resource": { + "resourceType": "Observation", + "id": "8572f071-3410-4162-b2d4-4e549320795e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 67.19, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d8044db-9081-4bc2-988e-9f5c54fbdc97", + "resource": { + "resourceType": "Observation", + "id": "5d8044db-9081-4bc2-988e-9f5c54fbdc97", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 14.93, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8bc31c89-5920-4d8f-8bb2-b9a23400b192", + "resource": { + "resourceType": "Observation", + "id": "8bc31c89-5920-4d8f-8bb2-b9a23400b192", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:abda425b-5dcf-4e5b-ab29-671cc5eb4a76", + "resource": { + "resourceType": "Observation", + "id": "abda425b-5dcf-4e5b-ab29-671cc5eb4a76", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.8, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4d938e2f-eafd-452d-8683-2b657dc7c8ac", + "resource": { + "resourceType": "Observation", + "id": "4d938e2f-eafd-452d-8683-2b657dc7c8ac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 138.61, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:04d489bd-f171-4c6b-9f37-141934d09762", + "resource": { + "resourceType": "Observation", + "id": "04d489bd-f171-4c6b-9f37-141934d09762", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.08, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4cce0680-8bac-4b52-8b85-9ada6816eda4", + "resource": { + "resourceType": "Observation", + "id": "4cce0680-8bac-4b52-8b85-9ada6816eda4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 109.63, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:20ea4432-3403-4331-b6e2-d8a579c43240", + "resource": { + "resourceType": "Observation", + "id": "20ea4432-3403-4331-b6e2-d8a579c43240", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 28.45, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ba5431d-2b30-4cba-930c-0d36e31b3cb7", + "resource": { + "resourceType": "Observation", + "id": "6ba5431d-2b30-4cba-930c-0d36e31b3cb7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:85ebea82-7fc5-4775-afd1-ef0d2b801ae1", + "resource": { + "resourceType": "Immunization", + "id": "85ebea82-7fc5-4775-afd1-ef0d2b801ae1", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "occurrenceDateTime": "2018-01-01T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7d95bd33-c7e3-479e-949d-323110979ba7", + "resource": { + "resourceType": "Immunization", + "id": "7d95bd33-c7e3-479e-949d-323110979ba7", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "occurrenceDateTime": "2018-01-01T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:114dfa31-82b4-4564-a64e-0c94af7c88c0", + "resource": { + "resourceType": "DiagnosticReport", + "id": "114dfa31-82b4-4564-a64e-0c94af7c88c0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:8572f071-3410-4162-b2d4-4e549320795e", + "display": "Glucose" + }, + { + "reference": "urn:uuid:5d8044db-9081-4bc2-988e-9f5c54fbdc97", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:8bc31c89-5920-4d8f-8bb2-b9a23400b192", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:abda425b-5dcf-4e5b-ab29-671cc5eb4a76", + "display": "Calcium" + }, + { + "reference": "urn:uuid:4d938e2f-eafd-452d-8683-2b657dc7c8ac", + "display": "Sodium" + }, + { + "reference": "urn:uuid:04d489bd-f171-4c6b-9f37-141934d09762", + "display": "Potassium" + }, + { + "reference": "urn:uuid:4cce0680-8bac-4b52-8b85-9ada6816eda4", + "display": "Chloride" + }, + { + "reference": "urn:uuid:20ea4432-3403-4331-b6e2-d8a579c43240", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:bb5824dd-797a-4f24-946d-a7e710e3a391", + "resource": { + "resourceType": "Claim", + "id": "bb5824dd-797a-4f24-946d-a7e710e3a391", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "created": "2018-01-01T04:36:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:85ebea82-7fc5-4775-afd1-ef0d2b801ae1" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7d95bd33-c7e3-479e-949d-323110979ba7" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:96cc857a-9127-43f9-829f-1ba3ad1af8ec", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "96cc857a-9127-43f9-829f-1ba3ad1af8ec", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bb5824dd-797a-4f24-946d-a7e710e3a391" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2018-01-01T04:36:42-08:00", + "end": "2019-01-01T04:36:42-08:00" + }, + "created": "2018-01-01T04:36:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:bb5824dd-797a-4f24-946d-a7e710e3a391" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 224.83200000000002, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e", + "resource": { + "resourceType": "Encounter", + "id": "576347df-eecc-4bef-b430-8117181dcd6e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "individual": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "display": "Dr. Franklin857 Tillman293" + } + } + ], + "period": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:33355ff7-c208-4cd5-a114-da4d67285f20", + "resource": { + "resourceType": "Observation", + "id": "33355ff7-c208-4cd5-a114-da4d67285f20", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" + }, + "effectiveDateTime": "2018-01-01T04:21:42-08:00", + "issued": "2018-01-01T04:21:42.840-08:00", + "valueQuantity": { + "value": 5.97, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:310eae17-0e5b-421e-bf42-53765954b278", + "resource": { + "resourceType": "Procedure", + "id": "310eae17-0e5b-421e-bf42-53765954b278", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" + }, + "performedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:04:42-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:79bc6b37-fd91-4870-995c-d931221e0652", + "resource": { + "resourceType": "Procedure", + "id": "79bc6b37-fd91-4870-995c-d931221e0652", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" + }, + "performedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T04:36:42-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d370258b-0a72-47dd-b762-dd61b29dcd63", + "resource": { + "resourceType": "Claim", + "id": "d370258b-0a72-47dd-b762-dd61b29dcd63", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "created": "2018-01-01T05:19:42-08:00", + "provider": { + "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "display": "LOWELL GENERAL HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:310eae17-0e5b-421e-bf42-53765954b278" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:79bc6b37-fd91-4870-995c-d931221e0652" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 9754.14, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 529.07, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1bca55b3-5f6b-4168-aa47-cdf51a576d98", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1bca55b3-5f6b-4168-aa47-cdf51a576d98", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "performer": [ + { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d370258b-0a72-47dd-b762-dd61b29dcd63" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2018-01-01T05:19:42-08:00", + "end": "2019-01-01T05:19:42-08:00" + }, + "created": "2018-01-01T05:19:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:d370258b-0a72-47dd-b762-dd61b29dcd63" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 9754.14, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1950.828, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7803.312, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 9754.14, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 9754.14, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2018-01-01T04:21:42-08:00", + "end": "2018-01-01T05:19:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 529.07, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 105.81400000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 423.2560000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 529.07, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 529.07, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 8226.568, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1", + "resource": { + "resourceType": "Encounter", + "id": "a721f251-d45e-40c1-8da1-1edf0b8e92f1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:07719b28-0951-48e4-a494-f720fc508653", + "resource": { + "resourceType": "Observation", + "id": "07719b28-0951-48e4-a494-f720fc508653", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfe107b7-8e45-47e3-a30c-aab5562c2915", + "resource": { + "resourceType": "Observation", + "id": "cfe107b7-8e45-47e3-a30c-aab5562c2915", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28ffe479-e81b-4e3f-9349-9d911afdd5ba", + "resource": { + "resourceType": "Observation", + "id": "28ffe479-e81b-4e3f-9349-9d911afdd5ba", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 89.2, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fe5259ca-c339-4876-bfdf-42c80a79eaf7", + "resource": { + "resourceType": "Observation", + "id": "fe5259ca-c339-4876-bfdf-42c80a79eaf7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 29.19, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2741101d-771b-45cc-a700-6c0246e183f5", + "resource": { + "resourceType": "Observation", + "id": "2741101d-771b-45cc-a700-6c0246e183f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:156e6b08-063e-43bf-ac47-06df7bb1027b", + "resource": { + "resourceType": "Observation", + "id": "156e6b08-063e-43bf-ac47-06df7bb1027b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 67, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c93dde1e-33f8-421b-8b95-85f266916b27", + "resource": { + "resourceType": "Observation", + "id": "c93dde1e-33f8-421b-8b95-85f266916b27", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2f00950-67ef-43e2-899d-790510611905", + "resource": { + "resourceType": "Observation", + "id": "e2f00950-67ef-43e2-899d-790510611905", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 82.75, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:63fe2f46-9f03-42a9-9eff-29721a754a1d", + "resource": { + "resourceType": "Observation", + "id": "63fe2f46-9f03-42a9-9eff-29721a754a1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 7.22, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c1d13db-9b7f-4552-b39a-16d45483dc75", + "resource": { + "resourceType": "Observation", + "id": "0c1d13db-9b7f-4552-b39a-16d45483dc75", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.15, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3dad7698-39da-4ce9-af7a-3da4687945b9", + "resource": { + "resourceType": "Observation", + "id": "3dad7698-39da-4ce9-af7a-3da4687945b9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.84, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fedd1f46-d752-4ac8-8d7f-fe5059d941ef", + "resource": { + "resourceType": "Observation", + "id": "fedd1f46-d752-4ac8-8d7f-fe5059d941ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 140.73, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:448f2d10-75e9-4431-b73b-6237dead796b", + "resource": { + "resourceType": "Observation", + "id": "448f2d10-75e9-4431-b73b-6237dead796b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.93, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", + "resource": { + "resourceType": "Observation", + "id": "6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 106.98, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab276e3d-4f4e-495b-be7f-b3e9287a752b", + "resource": { + "resourceType": "Observation", + "id": "ab276e3d-4f4e-495b-be7f-b3e9287a752b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 27.25, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bc99199a-eee1-499e-8bf0-76ab223ae204", + "resource": { + "resourceType": "Observation", + "id": "bc99199a-eee1-499e-8bf0-76ab223ae204", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:195fbf1b-90f3-4c1d-9785-139d3b4524ff", + "resource": { + "resourceType": "Observation", + "id": "195fbf1b-90f3-4c1d-9785-139d3b4524ff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "valueQuantity": { + "value": 5.83, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d0668a8-2b39-4a87-8c13-0bbf20a09d04", + "resource": { + "resourceType": "Procedure", + "id": "9d0668a8-2b39-4a87-8c13-0bbf20a09d04", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "performedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:36:42-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:930c7097-a336-4d01-9c6b-ff038011ffe1", + "resource": { + "resourceType": "Immunization", + "id": "930c7097-a336-4d01-9c6b-ff038011ffe1", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "occurrenceDateTime": "2019-01-07T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:22e25df5-de3c-45da-959d-24a25a8553d1", + "resource": { + "resourceType": "Immunization", + "id": "22e25df5-de3c-45da-959d-24a25a8553d1", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "occurrenceDateTime": "2019-01-07T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1959d580-9164-4a51-84cf-a374c3b97d22", + "resource": { + "resourceType": "Immunization", + "id": "1959d580-9164-4a51-84cf-a374c3b97d22", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "occurrenceDateTime": "2019-01-07T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c8799c64-5ebd-4133-8cfa-b521abaa6387", + "resource": { + "resourceType": "DiagnosticReport", + "id": "c8799c64-5ebd-4133-8cfa-b521abaa6387", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + }, + "effectiveDateTime": "2019-01-07T04:21:42-08:00", + "issued": "2019-01-07T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:e2f00950-67ef-43e2-899d-790510611905", + "display": "Glucose" + }, + { + "reference": "urn:uuid:63fe2f46-9f03-42a9-9eff-29721a754a1d", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:0c1d13db-9b7f-4552-b39a-16d45483dc75", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:3dad7698-39da-4ce9-af7a-3da4687945b9", + "display": "Calcium" + }, + { + "reference": "urn:uuid:fedd1f46-d752-4ac8-8d7f-fe5059d941ef", + "display": "Sodium" + }, + { + "reference": "urn:uuid:448f2d10-75e9-4431-b73b-6237dead796b", + "display": "Potassium" + }, + { + "reference": "urn:uuid:6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", + "display": "Chloride" + }, + { + "reference": "urn:uuid:ab276e3d-4f4e-495b-be7f-b3e9287a752b", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0f9bc1f1-05bc-4172-8827-bc8fa6c7e353", + "resource": { + "resourceType": "Claim", + "id": "0f9bc1f1-05bc-4172-8827-bc8fa6c7e353", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "created": "2019-01-07T04:51:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:930c7097-a336-4d01-9c6b-ff038011ffe1" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:22e25df5-de3c-45da-959d-24a25a8553d1" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1959d580-9164-4a51-84cf-a374c3b97d22" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:9d0668a8-2b39-4a87-8c13-0bbf20a09d04" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 5, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 505.09, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8055cd60-5b93-4bf1-a7c7-c2d0e0d65eea", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8055cd60-5b93-4bf1-a7c7-c2d0e0d65eea", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0f9bc1f1-05bc-4172-8827-bc8fa6c7e353" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2019-01-07T04:51:42-08:00", + "end": "2020-01-07T04:51:42-08:00" + }, + "created": "2019-01-07T04:51:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0f9bc1f1-05bc-4172-8827-bc8fa6c7e353" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "servicedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2019-01-07T04:21:42-08:00", + "end": "2019-01-07T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 505.09, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 101.018, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 404.072, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 505.09, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 505.09, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 741.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896", + "resource": { + "resourceType": "Encounter", + "id": "c9dd0522-d214-44e6-84a4-30e2751f1896", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Mr. Emerson869 Prohaska837" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "individual": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "display": "Dr. Lillia547 Gerlach374" + } + } + ], + "period": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0a8e1b96-88a3-49cf-97d9-5316f8dad8ff", + "resource": { + "resourceType": "Observation", + "id": "0a8e1b96-88a3-49cf-97d9-5316f8dad8ff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 174.8, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2aabc75c-abf4-4453-9248-a0307713d1ae", + "resource": { + "resourceType": "Observation", + "id": "2aabc75c-abf4-4453-9248-a0307713d1ae", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1502e9d5-4125-42b5-9847-77319e2551a5", + "resource": { + "resourceType": "Observation", + "id": "1502e9d5-4125-42b5-9847-77319e2551a5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 91.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32c9d0d2-c8a5-4e3f-8755-50168d5dd802", + "resource": { + "resourceType": "Observation", + "id": "32c9d0d2-c8a5-4e3f-8755-50168d5dd802", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 29.95, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51bff806-dbe9-4816-ba5b-788986cd06cd", + "resource": { + "resourceType": "Observation", + "id": "51bff806-dbe9-4816-ba5b-788986cd06cd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 116, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2793e50e-36fa-471a-b94b-b2c07b48d352", + "resource": { + "resourceType": "Observation", + "id": "2793e50e-36fa-471a-b94b-b2c07b48d352", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 90, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2315a08e-491b-401f-831e-6343392eb258", + "resource": { + "resourceType": "Observation", + "id": "2315a08e-491b-401f-831e-6343392eb258", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc4daf50-6cfa-4650-9e75-8f333ecec4d1", + "resource": { + "resourceType": "Observation", + "id": "dc4daf50-6cfa-4650-9e75-8f333ecec4d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 96.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:27b36e11-a573-4105-80f7-4264fb24b297", + "resource": { + "resourceType": "Observation", + "id": "27b36e11-a573-4105-80f7-4264fb24b297", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 10.81, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8d02f82f-f29a-4a9e-8757-c1fed998352f", + "resource": { + "resourceType": "Observation", + "id": "8d02f82f-f29a-4a9e-8757-c1fed998352f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 1.12, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7455a716-3c97-4b44-818e-4202ef3cd1dd", + "resource": { + "resourceType": "Observation", + "id": "7455a716-3c97-4b44-818e-4202ef3cd1dd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 9.33, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", + "resource": { + "resourceType": "Observation", + "id": "fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 140.64, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", + "resource": { + "resourceType": "Observation", + "id": "1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.76, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d6fb2d0d-a442-472e-a7f0-691a968bf4f0", + "resource": { + "resourceType": "Observation", + "id": "d6fb2d0d-a442-472e-a7f0-691a968bf4f0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 105.18, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b709490-a933-427f-8580-1db7d10b95a1", + "resource": { + "resourceType": "Observation", + "id": "8b709490-a933-427f-8580-1db7d10b95a1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 26.06, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddc1d4aa-8010-43a1-a764-6d0385671973", + "resource": { + "resourceType": "Observation", + "id": "ddc1d4aa-8010-43a1-a764-6d0385671973", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.7327, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d70bb003-0c32-4d7a-af9f-3374f3db3da1", + "resource": { + "resourceType": "Observation", + "id": "d70bb003-0c32-4d7a-af9f-3374f3db3da1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 4.1641, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67c95ca7-4af0-4c5f-aea0-a4a943297f65", + "resource": { + "resourceType": "Observation", + "id": "67c95ca7-4af0-4c5f-aea0-a4a943297f65", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 13.07, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:755d1096-d7b0-430a-bce1-ea660114a66e", + "resource": { + "resourceType": "Observation", + "id": "755d1096-d7b0-430a-bce1-ea660114a66e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 39.996, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", + "resource": { + "resourceType": "Observation", + "id": "ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 89.485, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e6af9c0-c3fe-4eaf-b609-710737833383", + "resource": { + "resourceType": "Observation", + "id": "1e6af9c0-c3fe-4eaf-b609-710737833383", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 31.815, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a628ef78-13b8-4279-9909-efe174ac51b3", + "resource": { + "resourceType": "Observation", + "id": "a628ef78-13b8-4279-9909-efe174ac51b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 34.573, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70d77739-25c0-4ce1-8196-55e4feea73d2", + "resource": { + "resourceType": "Observation", + "id": "70d77739-25c0-4ce1-8196-55e4feea73d2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 45.279, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", + "resource": { + "resourceType": "Observation", + "id": "ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 183.14, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b81eca3-6be0-462b-9650-5152a567bf2a", + "resource": { + "resourceType": "Observation", + "id": "1b81eca3-6be0-462b-9650-5152a567bf2a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 239.92, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8ae6ef1d-78ab-4655-9fba-2f4954b53f48", + "resource": { + "resourceType": "Observation", + "id": "8ae6ef1d-78ab-4655-9fba-2f4954b53f48", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 11.424, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bfa3ab3a-3ac8-4c8c-b061-b831c8980c3b", + "resource": { + "resourceType": "Observation", + "id": "bfa3ab3a-3ac8-4c8c-b061-b831c8980c3b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31e62da5-29f6-4120-908c-2d55a557bb04", + "resource": { + "resourceType": "Observation", + "id": "31e62da5-29f6-4120-908c-2d55a557bb04", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "valueQuantity": { + "value": 5.87, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d3d6b965-7193-4a81-93ae-1bc4539f6c06", + "resource": { + "resourceType": "Procedure", + "id": "d3d6b965-7193-4a81-93ae-1bc4539f6c06", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "performedPeriod": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:36:42-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a2658308-c442-4a5a-84ca-682a542f56aa", + "resource": { + "resourceType": "Immunization", + "id": "a2658308-c442-4a5a-84ca-682a542f56aa", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "occurrenceDateTime": "2020-01-13T04:21:42-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:9382b897-111d-4662-a591-e8b130d21816", + "resource": { + "resourceType": "DiagnosticReport", + "id": "9382b897-111d-4662-a591-e8b130d21816", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:dc4daf50-6cfa-4650-9e75-8f333ecec4d1", + "display": "Glucose" + }, + { + "reference": "urn:uuid:27b36e11-a573-4105-80f7-4264fb24b297", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:8d02f82f-f29a-4a9e-8757-c1fed998352f", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:7455a716-3c97-4b44-818e-4202ef3cd1dd", + "display": "Calcium" + }, + { + "reference": "urn:uuid:fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", + "display": "Sodium" + }, + { + "reference": "urn:uuid:1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", + "display": "Potassium" + }, + { + "reference": "urn:uuid:d6fb2d0d-a442-472e-a7f0-691a968bf4f0", + "display": "Chloride" + }, + { + "reference": "urn:uuid:8b709490-a933-427f-8580-1db7d10b95a1", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:66428479-9d74-4926-a182-c74b1a841a0f", + "resource": { + "resourceType": "DiagnosticReport", + "id": "66428479-9d74-4926-a182-c74b1a841a0f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "encounter": { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + }, + "effectiveDateTime": "2020-01-13T04:21:42-08:00", + "issued": "2020-01-13T04:21:42.840-08:00", + "result": [ + { + "reference": "urn:uuid:ddc1d4aa-8010-43a1-a764-6d0385671973", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:d70bb003-0c32-4d7a-af9f-3374f3db3da1", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:67c95ca7-4af0-4c5f-aea0-a4a943297f65", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:755d1096-d7b0-430a-bce1-ea660114a66e", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:1e6af9c0-c3fe-4eaf-b609-710737833383", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:a628ef78-13b8-4279-9909-efe174ac51b3", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:70d77739-25c0-4ce1-8196-55e4feea73d2", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1b81eca3-6be0-462b-9650-5152a567bf2a", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:8ae6ef1d-78ab-4655-9fba-2f4954b53f48", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:a0c609cd-ca62-45d8-8051-54c4316c4a8a", + "resource": { + "resourceType": "Claim", + "id": "a0c609cd-ca62-45d8-8051-54c4316c4a8a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", + "display": "Emerson869 Prohaska837" + }, + "billablePeriod": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "created": "2020-01-13T04:51:42-08:00", + "provider": { + "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a2658308-c442-4a5a-84ca-682a542f56aa" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:d3d6b965-7193-4a81-93ae-1bc4539f6c06" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 991.58, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bbe8a13e-8315-4a7f-b51e-bd4884fd1d38", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "bbe8a13e-8315-4a7f-b51e-bd4884fd1d38", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "requester": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "performer": [ + { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a0c609cd-ca62-45d8-8051-54c4316c4a8a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" + }, + "billablePeriod": { + "start": "2020-01-13T04:51:42-08:00", + "end": "2021-01-13T04:51:42-08:00" + }, + "created": "2020-01-13T04:51:42-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a0c609cd-ca62-45d8-8051-54c4316c4a8a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2020-01-13T04:21:42-08:00", + "end": "2020-01-13T04:51:42-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 991.58, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 198.31600000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 793.2640000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 991.58, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 991.58, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 905.6800000000002, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json new file mode 100644 index 000000000000..901fdf2e3177 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json @@ -0,0 +1,31841 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "resource": { + "resourceType": "Patient", + "id": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7730420515391289066 Population seed: 1586368870505
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Magnolia736 Weber641" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", + "valueAddress": { + "city": "Concord", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 5.141867221161322 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 53.85813277883868 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "SS", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-30-3400" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99966155" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "PPN", + "display": "Passport Number" + } + ], + "text": "Passport Number" + }, + "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", + "value": "X73794696X" + } + ], + "name": [ + { + "use": "official", + "family": "Marvin195", + "given": [ + "Lorette239" + ], + "prefix": [ + "Mrs." + ] + }, + { + "use": "maiden", + "family": "Zemlak964", + "given": [ + "Lorette239" + ], + "prefix": [ + "Mrs." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-259-1705", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1960-10-02", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.378055340679865 + }, + { + "url": "longitude", + "valueDecimal": -71.01059430318017 + } + ] + } + ], + "line": [ + "755 Von Mission" + ], + "city": "Boston", + "state": "Massachusetts", + "postalCode": "02113", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", + "code": "M", + "display": "M" + } + ], + "text": "M" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "resource": { + "resourceType": "Organization", + "id": "76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "76a80350-916e-3fc9-8fed-2c8c4b36c42f" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC.", + "telecom": [ + { + "system": "phone", + "value": "617-912-7900" + } + ], + "address": [ + { + "line": [ + "301 BROADWAY" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "02150-2807", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "resource": { + "resourceType": "Practitioner", + "id": "0390078a-1894-335b-9136-5f518ee1dbb6", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999931799" + } + ], + "active": true, + "name": [ + { + "family": "Koelpin146", + "given": [ + "Cheryle584" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Cheryle584.Koelpin146@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "301 BROADWAY" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "02150-2807", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7", + "resource": { + "resourceType": "Encounter", + "id": "f2ae7ab9-8abe-424a-a52b-72d3603542b7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1982-12-05T22:43:39-08:00", + "end": "1982-12-05T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "1982-12-05T22:43:39-08:00", + "end": "1982-12-05T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c", + "resource": { + "resourceType": "Condition", + "id": "788dd653-78c0-4922-a4ec-fcd8f720e60c", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" + }, + "onsetDateTime": "1982-12-05T22:43:39-08:00", + "recordedDate": "1982-12-05T22:43:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:bf763ed9-87a4-4aa4-853e-7aa252cbec05", + "resource": { + "resourceType": "CareTeam", + "id": "bf763ed9-87a4-4aa4-853e-7aa252cbec05", + "status": "active", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" + }, + "period": { + "start": "1982-12-05T22:43:39-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:927b39fe-945e-4b26-961e-c856bcadc2fb", + "resource": { + "resourceType": "CarePlan", + "id": "927b39fe-945e-4b26-961e-c856bcadc2fb", + "text": { + "status": "generated", + "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698360004", + "display": "Diabetes self management plan" + } + ], + "text": "Diabetes self management plan" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" + }, + "period": { + "start": "1982-12-05T22:43:39-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:bf763ed9-87a4-4aa4-853e-7aa252cbec05" + } + ], + "addresses": [ + { + "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "160670007", + "display": "Diabetic diet" + } + ], + "text": "Diabetic diet" + }, + "status": "in-progress", + "location": { + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "location": { + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:6e42015f-6507-409b-a2ba-08cba73365f4", + "resource": { + "resourceType": "Claim", + "id": "6e42015f-6507-409b-a2ba-08cba73365f4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "1982-12-05T22:43:39-08:00", + "end": "1982-12-05T23:13:39-08:00" + }, + "created": "1982-12-05T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cd3adb27-2387-401c-b16e-152e4a914e40", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cd3adb27-2387-401c-b16e-152e4a914e40", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6e42015f-6507-409b-a2ba-08cba73365f4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "1982-12-05T23:13:39-08:00", + "end": "1983-12-05T23:13:39-08:00" + }, + "created": "1982-12-05T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:6e42015f-6507-409b-a2ba-08cba73365f4" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1982-12-05T22:43:39-08:00", + "end": "1982-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "servicedPeriod": { + "start": "1982-12-05T22:43:39-08:00", + "end": "1982-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14", + "resource": { + "resourceType": "Encounter", + "id": "5b0e5a93-b152-4a67-8cfc-05fa6ef49a14", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1985-12-08T22:43:39-08:00", + "end": "1985-12-08T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "1985-12-08T22:43:39-08:00", + "end": "1985-12-08T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba", + "resource": { + "resourceType": "Condition", + "id": "8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" + }, + "onsetDateTime": "1985-12-08T22:43:39-08:00", + "recordedDate": "1985-12-08T22:43:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:444b38a1-5335-42ad-a80d-6a02e9fe245f", + "resource": { + "resourceType": "Claim", + "id": "444b38a1-5335-42ad-a80d-6a02e9fe245f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "1985-12-08T22:43:39-08:00", + "end": "1985-12-08T23:13:39-08:00" + }, + "created": "1985-12-08T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Aetna" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:383d050d-44e4-42a7-98fd-969541d92aa0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "383d050d-44e4-42a7-98fd-969541d92aa0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Aetna" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Aetna" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "444b38a1-5335-42ad-a80d-6a02e9fe245f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "1985-12-08T23:13:39-08:00", + "end": "1986-12-08T23:13:39-08:00" + }, + "created": "1985-12-08T23:13:39-08:00", + "insurer": { + "display": "Aetna" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:444b38a1-5335-42ad-a80d-6a02e9fe245f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Aetna" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1985-12-08T22:43:39-08:00", + "end": "1985-12-08T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "271737000", + "display": "Anemia (disorder)" + } + ], + "text": "Anemia (disorder)" + }, + "servicedPeriod": { + "start": "1985-12-08T22:43:39-08:00", + "end": "1985-12-08T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "resource": { + "resourceType": "Organization", + "id": "d733d4a9-080d-3593-b910-2366e652b7ea", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d733d4a9-080d-3593-b910-2366e652b7ea" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6179837000" + } + ], + "address": [ + { + "line": [ + "1153 CENTRE STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02130", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "resource": { + "resourceType": "Practitioner", + "id": "3a576225-3a87-3704-a672-e1c8bc03389f", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999519" + } + ], + "active": true, + "name": [ + { + "family": "Howell947", + "given": [ + "Max124" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Max124.Howell947@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "1153 CENTRE STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02130", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6", + "resource": { + "resourceType": "Encounter", + "id": "52ff6ac0-6eaf-45ec-869a-60a846973ab6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1986-10-26T22:43:39-08:00", + "end": "1986-10-26T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "display": "Dr. Max124 Howell947" + } + } + ], + "period": { + "start": "1986-10-26T22:43:39-08:00", + "end": "1986-10-26T23:28:39-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6", + "resource": { + "resourceType": "Condition", + "id": "c2003678-0785-4395-bdb1-0311e15128c6", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + ], + "text": "Miscarriage in first trimester" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" + }, + "onsetDateTime": "1986-10-26T22:43:39-08:00", + "recordedDate": "1986-10-26T22:43:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45", + "resource": { + "resourceType": "Claim", + "id": "6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "1986-10-26T22:43:39-08:00", + "end": "1986-10-26T23:28:39-08:00" + }, + "created": "1986-10-26T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + }, + "encounter": [ + { + "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + ], + "text": "Miscarriage in first trimester" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cd9388fe-54d7-4057-8ade-975d7457b917", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cd9388fe-54d7-4057-8ade-975d7457b917", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "performer": [ + { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "1986-10-26T23:28:39-08:00", + "end": "1987-10-26T23:28:39-08:00" + }, + "created": "1986-10-26T23:28:39-08:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + }, + "servicedPeriod": { + "start": "1986-10-26T22:43:39-08:00", + "end": "1986-10-26T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "19169002", + "display": "Miscarriage in first trimester" + } + ], + "text": "Miscarriage in first trimester" + }, + "servicedPeriod": { + "start": "1986-10-26T22:43:39-08:00", + "end": "1986-10-26T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424", + "resource": { + "resourceType": "Encounter", + "id": "2ccc9f08-6dee-48fe-9b62-a7aed300e424", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1997-12-21T22:43:39-08:00", + "end": "1997-12-21T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "1997-12-21T22:43:39-08:00", + "end": "1997-12-21T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "resource": { + "resourceType": "Condition", + "id": "f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "87433001", + "display": "Pulmonary emphysema (disorder)" + } + ], + "text": "Pulmonary emphysema (disorder)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" + }, + "onsetDateTime": "1997-12-21T22:43:39-08:00", + "recordedDate": "1997-12-21T22:43:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:461a5652-6d82-4e33-b0a6-44229bea93ed", + "resource": { + "resourceType": "CareTeam", + "id": "461a5652-6d82-4e33-b0a6-44229bea93ed", + "status": "active", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" + }, + "period": { + "start": "1997-12-21T22:43:39-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "87433001", + "display": "Pulmonary emphysema (disorder)" + } + ], + "text": "Pulmonary emphysema (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:ab5480fe-a33d-4188-9a89-f01ce663f0fd", + "resource": { + "resourceType": "CarePlan", + "id": "ab5480fe-a33d-4188-9a89-f01ce663f0fd", + "text": { + "status": "generated", + "div": "
    Care Plan for Chronic obstructive pulmonary disease clinical management plan.
    Activities:
    • Chronic obstructive pulmonary disease clinical management plan
    • Chronic obstructive pulmonary disease clinical management plan

    Care plan is meant to treat Pulmonary emphysema (disorder).
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "412776001", + "display": "Chronic obstructive pulmonary disease clinical management plan" + } + ], + "text": "Chronic obstructive pulmonary disease clinical management plan" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" + }, + "period": { + "start": "1997-12-21T22:43:39-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:461a5652-6d82-4e33-b0a6-44229bea93ed" + } + ], + "addresses": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "location": { + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15081005", + "display": "Pulmonary rehabilitation (regime/therapy)" + } + ], + "text": "Pulmonary rehabilitation (regime/therapy)" + }, + "status": "in-progress", + "location": { + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:7341a535-24a6-4871-8750-b71c03f22e4d", + "resource": { + "resourceType": "Claim", + "id": "7341a535-24a6-4871-8750-b71c03f22e4d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "1997-12-21T22:43:39-08:00", + "end": "1997-12-21T23:13:39-08:00" + }, + "created": "1997-12-21T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "87433001", + "display": "Pulmonary emphysema (disorder)" + } + ], + "text": "Pulmonary emphysema (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f07c5217-22d2-44cc-b327-07987b795cfa", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f07c5217-22d2-44cc-b327-07987b795cfa", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7341a535-24a6-4871-8750-b71c03f22e4d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "1997-12-21T23:13:39-08:00", + "end": "1998-12-21T23:13:39-08:00" + }, + "created": "1997-12-21T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7341a535-24a6-4871-8750-b71c03f22e4d" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1997-12-21T22:43:39-08:00", + "end": "1997-12-21T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "87433001", + "display": "Pulmonary emphysema (disorder)" + } + ], + "text": "Pulmonary emphysema (disorder)" + }, + "servicedPeriod": { + "start": "1997-12-21T22:43:39-08:00", + "end": "1997-12-21T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0", + "resource": { + "resourceType": "Encounter", + "id": "35491525-a5e9-4f41-bfad-d92a0933e8d0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4a454ca9-849d-4b11-b0b8-636211ec2dbe", + "resource": { + "resourceType": "MedicationRequest", + "id": "4a454ca9-849d-4b11-b0b8-636211ec2dbe", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + }, + "authoredOn": "2000-10-08T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ec0f9c39-9a97-4230-9716-c51212ec8458", + "resource": { + "resourceType": "Claim", + "id": "ec0f9c39-9a97-4230-9716-c51212ec8458", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "created": "2000-10-09T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:4a454ca9-849d-4b11-b0b8-636211ec2dbe" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + } + ] + } + ], + "total": { + "value": 47.02, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cd0998fb-703c-4227-9aee-55c95ba6c0eb", + "resource": { + "resourceType": "MedicationRequest", + "id": "cd0998fb-703c-4227-9aee-55c95ba6c0eb", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + }, + "authoredOn": "2000-10-08T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ca39d049-e07a-421b-9afb-3b15b612cdbc", + "resource": { + "resourceType": "Claim", + "id": "ca39d049-e07a-421b-9afb-3b15b612cdbc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "created": "2000-10-09T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:cd0998fb-703c-4227-9aee-55c95ba6c0eb" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + } + ] + } + ], + "total": { + "value": 39.99, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ce915626-94c5-463f-bd62-e8b72a7c6cbe", + "resource": { + "resourceType": "Claim", + "id": "ce915626-94c5-463f-bd62-e8b72a7c6cbe", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "created": "2000-10-09T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:068b8034-f184-4029-896c-85706cff3b92", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "068b8034-f184-4029-896c-85706cff3b92", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ce915626-94c5-463f-bd62-e8b72a7c6cbe" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2000-10-09T00:13:39-07:00", + "end": "2001-10-09T00:13:39-07:00" + }, + "created": "2000-10-09T00:13:39-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ce915626-94c5-463f-bd62-e8b72a7c6cbe" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2000-10-08T23:43:39-07:00", + "end": "2000-10-09T00:13:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96", + "resource": { + "resourceType": "Encounter", + "id": "95112402-82bc-4ab9-912f-607f28d5ee96", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2001-10-14T23:43:39-07:00", + "end": "2001-10-15T00:13:39-07:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2001-10-14T23:43:39-07:00", + "end": "2001-10-15T00:13:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6e25a07d-403c-4270-b798-cd9b40f29e0c", + "resource": { + "resourceType": "MedicationRequest", + "id": "6e25a07d-403c-4270-b798-cd9b40f29e0c", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" + }, + "authoredOn": "2001-10-14T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:6706dbc8-36af-490e-b04e-4146f04e7bb1", + "resource": { + "resourceType": "Claim", + "id": "6706dbc8-36af-490e-b04e-4146f04e7bb1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2001-10-14T23:43:39-07:00", + "end": "2001-10-15T00:13:39-07:00" + }, + "created": "2001-10-15T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:6e25a07d-403c-4270-b798-cd9b40f29e0c" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" + } + ] + } + ], + "total": { + "value": 10.29, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4e26781a-79c9-4b4b-bb04-0f2ba2e359e9", + "resource": { + "resourceType": "Claim", + "id": "4e26781a-79c9-4b4b-bb04-0f2ba2e359e9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2001-10-14T23:43:39-07:00", + "end": "2001-10-15T00:13:39-07:00" + }, + "created": "2001-10-15T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:19a1350d-a290-4470-9a11-b056fa08c14f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "19a1350d-a290-4470-9a11-b056fa08c14f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4e26781a-79c9-4b4b-bb04-0f2ba2e359e9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2001-10-15T00:13:39-07:00", + "end": "2002-10-15T00:13:39-07:00" + }, + "created": "2001-10-15T00:13:39-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4e26781a-79c9-4b4b-bb04-0f2ba2e359e9" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2001-10-14T23:43:39-07:00", + "end": "2001-10-15T00:13:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c", + "resource": { + "resourceType": "Encounter", + "id": "4007e9d7-ca83-4a4b-ba98-e2115d096c0c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2002-10-20T23:43:39-07:00", + "end": "2002-10-21T00:13:39-07:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2002-10-20T23:43:39-07:00", + "end": "2002-10-21T00:13:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2d35ad83-68c9-48d8-a364-df62c353b26b", + "resource": { + "resourceType": "MedicationRequest", + "id": "2d35ad83-68c9-48d8-a364-df62c353b26b", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" + }, + "authoredOn": "2002-10-20T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:08b60435-7cb7-4ca4-a235-9cbed724a63b", + "resource": { + "resourceType": "Claim", + "id": "08b60435-7cb7-4ca4-a235-9cbed724a63b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2002-10-20T23:43:39-07:00", + "end": "2002-10-21T00:13:39-07:00" + }, + "created": "2002-10-21T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:2d35ad83-68c9-48d8-a364-df62c353b26b" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" + } + ] + } + ], + "total": { + "value": 19.73, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1835b212-674d-4caa-a2b4-cab25b255294", + "resource": { + "resourceType": "Claim", + "id": "1835b212-674d-4caa-a2b4-cab25b255294", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2002-10-20T23:43:39-07:00", + "end": "2002-10-21T00:13:39-07:00" + }, + "created": "2002-10-21T00:13:39-07:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ee7c7505-fd48-4804-a86a-ab23e5cf3efa", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ee7c7505-fd48-4804-a86a-ab23e5cf3efa", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1835b212-674d-4caa-a2b4-cab25b255294" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2002-10-21T00:13:39-07:00", + "end": "2003-10-21T00:13:39-07:00" + }, + "created": "2002-10-21T00:13:39-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:1835b212-674d-4caa-a2b4-cab25b255294" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2002-10-20T23:43:39-07:00", + "end": "2002-10-21T00:13:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242", + "resource": { + "resourceType": "Encounter", + "id": "4a967f12-9da3-4ade-87e2-a2a668cd5242", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2003-10-26T22:43:39-08:00", + "end": "2003-10-26T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2003-10-26T22:43:39-08:00", + "end": "2003-10-26T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a6520d80-4d23-453a-8ab0-fb6ecc05a179", + "resource": { + "resourceType": "MedicationRequest", + "id": "a6520d80-4d23-453a-8ab0-fb6ecc05a179", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" + }, + "authoredOn": "2003-10-26T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:09a033a7-7010-4f3f-8781-cf2619dd12bc", + "resource": { + "resourceType": "Claim", + "id": "09a033a7-7010-4f3f-8781-cf2619dd12bc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2003-10-26T22:43:39-08:00", + "end": "2003-10-26T23:13:39-08:00" + }, + "created": "2003-10-26T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:a6520d80-4d23-453a-8ab0-fb6ecc05a179" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" + } + ] + } + ], + "total": { + "value": 15.01, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ff57c60e-c6fa-4084-a102-9d1ef4715aaa", + "resource": { + "resourceType": "Claim", + "id": "ff57c60e-c6fa-4084-a102-9d1ef4715aaa", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2003-10-26T22:43:39-08:00", + "end": "2003-10-26T23:13:39-08:00" + }, + "created": "2003-10-26T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1d4f3c3e-27ab-46ea-98ca-f8d6aa9a89d3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1d4f3c3e-27ab-46ea-98ca-f8d6aa9a89d3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ff57c60e-c6fa-4084-a102-9d1ef4715aaa" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2003-10-26T23:13:39-08:00", + "end": "2004-10-26T23:13:39-07:00" + }, + "created": "2003-10-26T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ff57c60e-c6fa-4084-a102-9d1ef4715aaa" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2003-10-26T22:43:39-08:00", + "end": "2003-10-26T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "resource": { + "resourceType": "Organization", + "id": "44bef9d3-91c2-3005-93e0-ccf436348ff0", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "44bef9d3-91c2-3005-93e0-ccf436348ff0" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MASSACHUSETTS EYE AND EAR INFIRMARY -", + "telecom": [ + { + "system": "phone", + "value": "6175237900" + } + ], + "address": [ + { + "line": [ + "243 CHARLES STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "resource": { + "resourceType": "Practitioner", + "id": "fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999689" + } + ], + "active": true, + "name": [ + { + "family": "Mayert710", + "given": [ + "Dovie983" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Dovie983.Mayert710@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "243 CHARLES STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4", + "resource": { + "resourceType": "Encounter", + "id": "55ed12fc-16f8-4088-bb03-a0129f0606b4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2003-12-28T22:43:39-08:00", + "end": "2003-12-28T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + } + ], + "period": { + "start": "2003-12-28T22:43:39-08:00", + "end": "2003-12-28T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f83ae010-a644-4ecd-9eea-199e3452f2f3", + "resource": { + "resourceType": "MedicationRequest", + "id": "f83ae010-a644-4ecd-9eea-199e3452f2f3", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" + }, + "authoredOn": "2003-12-28T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:44fb369c-f31c-40a0-92b2-7b1ad39c5f4e", + "resource": { + "resourceType": "Claim", + "id": "44fb369c-f31c-40a0-92b2-7b1ad39c5f4e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2003-12-28T22:43:39-08:00", + "end": "2003-12-28T23:13:39-08:00" + }, + "created": "2003-12-28T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:f83ae010-a644-4ecd-9eea-199e3452f2f3" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" + } + ] + } + ], + "total": { + "value": 29.28, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d88517ec-bb58-4b93-8173-2f21f4e45b64", + "resource": { + "resourceType": "Claim", + "id": "d88517ec-bb58-4b93-8173-2f21f4e45b64", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2003-12-28T22:43:39-08:00", + "end": "2003-12-28T23:13:39-08:00" + }, + "created": "2003-12-28T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6c92aff6-8546-45e1-bfcd-5e9d0819af6e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6c92aff6-8546-45e1-bfcd-5e9d0819af6e", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "performer": [ + { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d88517ec-bb58-4b93-8173-2f21f4e45b64" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2003-12-28T23:13:39-08:00", + "end": "2004-12-28T23:13:39-08:00" + }, + "created": "2003-12-28T23:13:39-08:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:d88517ec-bb58-4b93-8173-2f21f4e45b64" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "2003-12-28T22:43:39-08:00", + "end": "2003-12-28T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6", + "resource": { + "resourceType": "Encounter", + "id": "b691ff04-6df6-404b-8560-ea8adfdd64f6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2004-10-31T22:43:39-08:00", + "end": "2004-10-31T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2004-10-31T22:43:39-08:00", + "end": "2004-10-31T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6862a1eb-8d30-4b28-8e51-5f00397eaea4", + "resource": { + "resourceType": "MedicationRequest", + "id": "6862a1eb-8d30-4b28-8e51-5f00397eaea4", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" + }, + "authoredOn": "2004-10-31T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a509d9fd-9b8d-4ac8-b30d-42952b34ebb7", + "resource": { + "resourceType": "Claim", + "id": "a509d9fd-9b8d-4ac8-b30d-42952b34ebb7", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2004-10-31T22:43:39-08:00", + "end": "2004-10-31T23:13:39-08:00" + }, + "created": "2004-10-31T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:6862a1eb-8d30-4b28-8e51-5f00397eaea4" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" + } + ] + } + ], + "total": { + "value": 26.81, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a1f029a5-53ac-4d92-8e65-fc409aa0a4c5", + "resource": { + "resourceType": "Claim", + "id": "a1f029a5-53ac-4d92-8e65-fc409aa0a4c5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2004-10-31T22:43:39-08:00", + "end": "2004-10-31T23:13:39-08:00" + }, + "created": "2004-10-31T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3e555a99-17cb-4f42-b31f-f5a2ad3b4a4d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3e555a99-17cb-4f42-b31f-f5a2ad3b4a4d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a1f029a5-53ac-4d92-8e65-fc409aa0a4c5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2004-10-31T23:13:39-08:00", + "end": "2005-10-31T23:13:39-08:00" + }, + "created": "2004-10-31T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a1f029a5-53ac-4d92-8e65-fc409aa0a4c5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2004-10-31T22:43:39-08:00", + "end": "2004-10-31T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26", + "resource": { + "resourceType": "Encounter", + "id": "80ca511b-b8fb-4219-b33b-89d241e69d26", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2005-11-06T22:43:39-08:00", + "end": "2005-11-06T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2005-11-06T22:43:39-08:00", + "end": "2005-11-06T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:839732db-f78d-467c-a29c-3e7b6d9cf9be", + "resource": { + "resourceType": "MedicationRequest", + "id": "839732db-f78d-467c-a29c-3e7b6d9cf9be", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" + }, + "authoredOn": "2005-11-06T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:725d52e5-4a61-44b1-8009-1c09d7afe0db", + "resource": { + "resourceType": "Claim", + "id": "725d52e5-4a61-44b1-8009-1c09d7afe0db", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2005-11-06T22:43:39-08:00", + "end": "2005-11-06T23:13:39-08:00" + }, + "created": "2005-11-06T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:839732db-f78d-467c-a29c-3e7b6d9cf9be" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" + } + ] + } + ], + "total": { + "value": 25.32, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:56fc389f-cb27-4a04-8c74-c345ea700456", + "resource": { + "resourceType": "Claim", + "id": "56fc389f-cb27-4a04-8c74-c345ea700456", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2005-11-06T22:43:39-08:00", + "end": "2005-11-06T23:13:39-08:00" + }, + "created": "2005-11-06T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a39f4ff8-705b-488c-88c9-3a69a138908a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a39f4ff8-705b-488c-88c9-3a69a138908a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "56fc389f-cb27-4a04-8c74-c345ea700456" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2005-11-06T23:13:39-08:00", + "end": "2006-11-06T23:13:39-08:00" + }, + "created": "2005-11-06T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:56fc389f-cb27-4a04-8c74-c345ea700456" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2005-11-06T22:43:39-08:00", + "end": "2005-11-06T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280", + "resource": { + "resourceType": "Encounter", + "id": "42d5eaf7-1718-4646-884e-541c2a45b280", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2006-11-12T22:43:39-08:00", + "end": "2006-11-12T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2006-11-12T22:43:39-08:00", + "end": "2006-11-12T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:90e50881-0517-4120-b7bf-d71e3a9d5332", + "resource": { + "resourceType": "MedicationRequest", + "id": "90e50881-0517-4120-b7bf-d71e3a9d5332", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" + }, + "authoredOn": "2006-11-12T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:573f7735-70da-4176-9950-95cf26936807", + "resource": { + "resourceType": "Claim", + "id": "573f7735-70da-4176-9950-95cf26936807", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2006-11-12T22:43:39-08:00", + "end": "2006-11-12T23:28:39-08:00" + }, + "created": "2006-11-12T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:90e50881-0517-4120-b7bf-d71e3a9d5332" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" + } + ] + } + ], + "total": { + "value": 21.18, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7fce27cc-7d95-4ea5-b103-80dbdf37b26a", + "resource": { + "resourceType": "Claim", + "id": "7fce27cc-7d95-4ea5-b103-80dbdf37b26a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2006-11-12T22:43:39-08:00", + "end": "2006-11-12T23:28:39-08:00" + }, + "created": "2006-11-12T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6a7d6334-f9d8-4213-98a9-1a806966b62d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6a7d6334-f9d8-4213-98a9-1a806966b62d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7fce27cc-7d95-4ea5-b103-80dbdf37b26a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2006-11-12T23:28:39-08:00", + "end": "2007-11-12T23:28:39-08:00" + }, + "created": "2006-11-12T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7fce27cc-7d95-4ea5-b103-80dbdf37b26a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2006-11-12T22:43:39-08:00", + "end": "2006-11-12T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507", + "resource": { + "resourceType": "Encounter", + "id": "a5a881df-72cc-455d-980c-4a0f81459507", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2007-11-18T22:43:39-08:00", + "end": "2007-11-18T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2007-11-18T22:43:39-08:00", + "end": "2007-11-18T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:25cf1693-7aca-4df1-9680-aaa1dd71843f", + "resource": { + "resourceType": "MedicationRequest", + "id": "25cf1693-7aca-4df1-9680-aaa1dd71843f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" + }, + "authoredOn": "2007-11-18T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:01e64044-0466-4df2-a0d1-ae8347935aea", + "resource": { + "resourceType": "Claim", + "id": "01e64044-0466-4df2-a0d1-ae8347935aea", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2007-11-18T22:43:39-08:00", + "end": "2007-11-18T23:13:39-08:00" + }, + "created": "2007-11-18T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:25cf1693-7aca-4df1-9680-aaa1dd71843f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" + } + ] + } + ], + "total": { + "value": 27.87, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:272d2b52-bbff-4217-ae09-26e4d49640d0", + "resource": { + "resourceType": "Claim", + "id": "272d2b52-bbff-4217-ae09-26e4d49640d0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2007-11-18T22:43:39-08:00", + "end": "2007-11-18T23:13:39-08:00" + }, + "created": "2007-11-18T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3e0605f6-f4f5-4f5e-a54e-a1e1244c5665", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3e0605f6-f4f5-4f5e-a54e-a1e1244c5665", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "272d2b52-bbff-4217-ae09-26e4d49640d0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2007-11-18T23:13:39-08:00", + "end": "2008-11-18T23:13:39-08:00" + }, + "created": "2007-11-18T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:272d2b52-bbff-4217-ae09-26e4d49640d0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2007-11-18T22:43:39-08:00", + "end": "2007-11-18T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f", + "resource": { + "resourceType": "Encounter", + "id": "35ffd429-2918-4891-8b60-6daf8ffe0b8f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2008-11-23T22:43:39-08:00", + "end": "2008-11-23T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2008-11-23T22:43:39-08:00", + "end": "2008-11-23T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:36b60b89-0a3f-42ae-9165-158611b69849", + "resource": { + "resourceType": "MedicationRequest", + "id": "36b60b89-0a3f-42ae-9165-158611b69849", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" + }, + "authoredOn": "2008-11-23T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:4abf2798-93b8-411c-947b-40b3bd84fcf2", + "resource": { + "resourceType": "Claim", + "id": "4abf2798-93b8-411c-947b-40b3bd84fcf2", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2008-11-23T22:43:39-08:00", + "end": "2008-11-23T23:28:39-08:00" + }, + "created": "2008-11-23T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:36b60b89-0a3f-42ae-9165-158611b69849" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" + } + ] + } + ], + "total": { + "value": 19.67, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4e37441d-055c-4244-a285-456f0ca248d5", + "resource": { + "resourceType": "Claim", + "id": "4e37441d-055c-4244-a285-456f0ca248d5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2008-11-23T22:43:39-08:00", + "end": "2008-11-23T23:28:39-08:00" + }, + "created": "2008-11-23T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:42e4f0c9-5d77-4a4d-8803-8dcb0a89a9b3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "42e4f0c9-5d77-4a4d-8803-8dcb0a89a9b3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4e37441d-055c-4244-a285-456f0ca248d5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2008-11-23T23:28:39-08:00", + "end": "2009-11-23T23:28:39-08:00" + }, + "created": "2008-11-23T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4e37441d-055c-4244-a285-456f0ca248d5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2008-11-23T22:43:39-08:00", + "end": "2008-11-23T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a", + "resource": { + "resourceType": "Encounter", + "id": "92136260-8183-4568-aef5-a720f0a63b5a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2009-06-23T23:43:39-07:00", + "end": "2009-06-24T00:23:39-07:00" + }, + "individual": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + } + ], + "period": { + "start": "2009-06-23T23:43:39-07:00", + "end": "2009-06-24T00:23:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5cd2892a-869d-4830-bd0c-953cd2d617f2", + "resource": { + "resourceType": "MedicationRequest", + "id": "5cd2892a-869d-4830-bd0c-953cd2d617f2", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "389221", + "display": "Etonogestrel 68 MG Drug Implant" + } + ], + "text": "Etonogestrel 68 MG Drug Implant" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" + }, + "authoredOn": "2009-06-23T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:fb52d101-c808-4709-89b7-2f1f00552ecb", + "resource": { + "resourceType": "Claim", + "id": "fb52d101-c808-4709-89b7-2f1f00552ecb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2009-06-23T23:43:39-07:00", + "end": "2009-06-24T00:23:39-07:00" + }, + "created": "2009-06-24T00:23:39-07:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:5cd2892a-869d-4830-bd0c-953cd2d617f2" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "encounter": [ + { + "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" + } + ] + } + ], + "total": { + "value": 652.94, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e0efce36-aa03-4aa6-99de-935b709f57be", + "resource": { + "resourceType": "Claim", + "id": "e0efce36-aa03-4aa6-99de-935b709f57be", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2009-06-23T23:43:39-07:00", + "end": "2009-06-24T00:23:39-07:00" + }, + "created": "2009-06-24T00:23:39-07:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "encounter": [ + { + "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6822102c-86b2-45de-b764-980f29bdab65", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6822102c-86b2-45de-b764-980f29bdab65", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "performer": [ + { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e0efce36-aa03-4aa6-99de-935b709f57be" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2009-06-24T00:23:39-07:00", + "end": "2010-06-24T00:23:39-07:00" + }, + "created": "2009-06-24T00:23:39-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e0efce36-aa03-4aa6-99de-935b709f57be" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "servicedPeriod": { + "start": "2009-06-23T23:43:39-07:00", + "end": "2009-06-24T00:23:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54", + "resource": { + "resourceType": "Encounter", + "id": "ba1f1989-030a-479c-9738-68fdf9b6aa54", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2009-11-29T22:43:39-08:00", + "end": "2009-11-29T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2009-11-29T22:43:39-08:00", + "end": "2009-11-29T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b09fd23e-17cd-46f0-80bc-cf6bd818c6d4", + "resource": { + "resourceType": "MedicationRequest", + "id": "b09fd23e-17cd-46f0-80bc-cf6bd818c6d4", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" + }, + "authoredOn": "2009-11-29T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:9b847e62-0da5-4586-862d-d747fc6eb3cc", + "resource": { + "resourceType": "Claim", + "id": "9b847e62-0da5-4586-862d-d747fc6eb3cc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2009-11-29T22:43:39-08:00", + "end": "2009-11-29T23:13:39-08:00" + }, + "created": "2009-11-29T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:b09fd23e-17cd-46f0-80bc-cf6bd818c6d4" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" + } + ] + } + ], + "total": { + "value": 23.23, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b0d0b831-8639-468c-b7b3-30bccc9b30f1", + "resource": { + "resourceType": "Claim", + "id": "b0d0b831-8639-468c-b7b3-30bccc9b30f1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2009-11-29T22:43:39-08:00", + "end": "2009-11-29T23:13:39-08:00" + }, + "created": "2009-11-29T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fab05b3d-4335-40a7-b497-513ba32928bf", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "fab05b3d-4335-40a7-b497-513ba32928bf", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b0d0b831-8639-468c-b7b3-30bccc9b30f1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2009-11-29T23:13:39-08:00", + "end": "2010-11-29T23:13:39-08:00" + }, + "created": "2009-11-29T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b0d0b831-8639-468c-b7b3-30bccc9b30f1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2009-11-29T22:43:39-08:00", + "end": "2009-11-29T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d", + "resource": { + "resourceType": "Encounter", + "id": "3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2010-06-18T23:43:39-07:00", + "end": "2010-06-18T23:58:39-07:00" + }, + "individual": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + } + ], + "period": { + "start": "2010-06-18T23:43:39-07:00", + "end": "2010-06-18T23:58:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3e216f89-3bae-41c4-87d2-e5e5fdb2cc28", + "resource": { + "resourceType": "MedicationRequest", + "id": "3e216f89-3bae-41c4-87d2-e5e5fdb2cc28", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "749762", + "display": "Seasonique 91 Day Pack" + } + ], + "text": "Seasonique 91 Day Pack" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" + }, + "authoredOn": "2010-06-18T23:43:39-07:00", + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0675548c-4bab-4e19-8aa2-730039a5d443", + "resource": { + "resourceType": "Claim", + "id": "0675548c-4bab-4e19-8aa2-730039a5d443", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2010-06-18T23:43:39-07:00", + "end": "2010-06-18T23:58:39-07:00" + }, + "created": "2010-06-18T23:58:39-07:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:3e216f89-3bae-41c4-87d2-e5e5fdb2cc28" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "encounter": [ + { + "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" + } + ] + } + ], + "total": { + "value": 31.88, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d27c0519-49cc-47ea-b498-32ef083873b9", + "resource": { + "resourceType": "Claim", + "id": "d27c0519-49cc-47ea-b498-32ef083873b9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2010-06-18T23:43:39-07:00", + "end": "2010-06-18T23:58:39-07:00" + }, + "created": "2010-06-18T23:58:39-07:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "encounter": [ + { + "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:64536aa6-cbfe-4bbc-9158-7727b725ce88", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "64536aa6-cbfe-4bbc-9158-7727b725ce88", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "performer": [ + { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d27c0519-49cc-47ea-b498-32ef083873b9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2010-06-18T23:58:39-07:00", + "end": "2011-06-18T23:58:39-07:00" + }, + "created": "2010-06-18T23:58:39-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:d27c0519-49cc-47ea-b498-32ef083873b9" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + }, + "servicedPeriod": { + "start": "2010-06-18T23:43:39-07:00", + "end": "2010-06-18T23:58:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5", + "resource": { + "resourceType": "Encounter", + "id": "0722e38b-0125-4f23-b29f-c619a66d4ef5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:31:39-07:00" + }, + "individual": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "display": "Dr. Max124 Howell947" + } + } + ], + "period": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:31:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b62a4fa4-c56f-4f8e-8270-7c17d118b636", + "resource": { + "resourceType": "Procedure", + "id": "b62a4fa4-c56f-4f8e-8270-7c17d118b636", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" + }, + "performedPeriod": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:16:39-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0", + "resource": { + "resourceType": "Claim", + "id": "1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:31:39-07:00" + }, + "created": "2010-10-04T00:31:39-07:00", + "provider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b62a4fa4-c56f-4f8e-8270-7c17d118b636" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 16816.16, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5946f3c3-56f0-492f-8859-cacefd6f53fb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5946f3c3-56f0-492f-8859-cacefd6f53fb", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "performer": [ + { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2010-10-04T00:31:39-07:00", + "end": "2011-10-04T00:31:39-07:00" + }, + "created": "2010-10-04T00:31:39-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:31:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2010-10-03T23:43:39-07:00", + "end": "2010-10-04T00:31:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 16816.16, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 3363.232, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 13452.928, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 16816.16, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 16816.16, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 13452.928, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524", + "resource": { + "resourceType": "Encounter", + "id": "b0e2b0cd-72e8-4b97-8dd2-6e388c01e524", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b884c0fd-e12f-49d0-a1ea-c8993d39dc32", + "resource": { + "resourceType": "Observation", + "id": "b884c0fd-e12f-49d0-a1ea-c8993d39dc32", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 67.379, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca0ceb79-a79f-4b69-8e94-9de20e94dc83", + "resource": { + "resourceType": "Observation", + "id": "ca0ceb79-a79f-4b69-8e94-9de20e94dc83", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9251ed3b-87b5-4b12-b152-7eb7871f87c5", + "resource": { + "resourceType": "Observation", + "id": "9251ed3b-87b5-4b12-b152-7eb7871f87c5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2a114b23-b9f8-4a43-bb3e-d865a9228ca0", + "resource": { + "resourceType": "Observation", + "id": "2a114b23-b9f8-4a43-bb3e-d865a9228ca0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 71.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8c9c5596-05a0-4f08-a7e9-59ac09237cb1", + "resource": { + "resourceType": "Observation", + "id": "8c9c5596-05a0-4f08-a7e9-59ac09237cb1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 27.52, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28aea52e-9ae7-4f24-a66f-836d63570dd4", + "resource": { + "resourceType": "Observation", + "id": "28aea52e-9ae7-4f24-a66f-836d63570dd4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 88, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 122, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5361305c-390f-4ef3-9093-0b08404cfdbd", + "resource": { + "resourceType": "Observation", + "id": "5361305c-390f-4ef3-9093-0b08404cfdbd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:10398e74-c8f3-4fa1-8938-37db79cf4581", + "resource": { + "resourceType": "Observation", + "id": "10398e74-c8f3-4fa1-8938-37db79cf4581", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7be083f3-8f42-40a8-9c36-1ca04255875f", + "resource": { + "resourceType": "Observation", + "id": "7be083f3-8f42-40a8-9c36-1ca04255875f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 86.79, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3777fdb4-4e26-441b-8760-59bf63aef0f0", + "resource": { + "resourceType": "Observation", + "id": "3777fdb4-4e26-441b-8760-59bf63aef0f0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a04df17d-795d-4971-b74d-605058b14505", + "resource": { + "resourceType": "Observation", + "id": "a04df17d-795d-4971-b74d-605058b14505", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29bc2948-0bed-40b6-9ac2-22e6129eef77", + "resource": { + "resourceType": "Observation", + "id": "29bc2948-0bed-40b6-9ac2-22e6129eef77", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 8.5, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:81f6ca85-5291-4753-8246-86bdfa0d8779", + "resource": { + "resourceType": "Observation", + "id": "81f6ca85-5291-4753-8246-86bdfa0d8779", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 139.14, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e251d63-421d-406e-8cfd-3d6c98fea52d", + "resource": { + "resourceType": "Observation", + "id": "8e251d63-421d-406e-8cfd-3d6c98fea52d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.81, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8f778cd4-7df9-472d-870e-25920094f962", + "resource": { + "resourceType": "Observation", + "id": "8f778cd4-7df9-472d-870e-25920094f962", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 108.07, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f7c5ce83-f870-42c0-9305-c627ef0a4128", + "resource": { + "resourceType": "Observation", + "id": "f7c5ce83-f870-42c0-9305-c627ef0a4128", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 26.58, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", + "resource": { + "resourceType": "Observation", + "id": "ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 168.67, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b24cd723-57ab-487c-aee1-6d613b11882b", + "resource": { + "resourceType": "Observation", + "id": "b24cd723-57ab-487c-aee1-6d613b11882b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 117.1, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:410421b0-f849-4ef7-ae7f-66e1fb94e5b8", + "resource": { + "resourceType": "Observation", + "id": "410421b0-f849-4ef7-ae7f-66e1fb94e5b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 83.17, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47c45751-fa57-45d4-a317-2ddbad218a1d", + "resource": { + "resourceType": "Observation", + "id": "47c45751-fa57-45d4-a317-2ddbad218a1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 62.07, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aa4b8ede-2213-4a33-a72a-f474629cc496", + "resource": { + "resourceType": "Observation", + "id": "aa4b8ede-2213-4a33-a72a-f474629cc496", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 7.575, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", + "resource": { + "resourceType": "Observation", + "id": "f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.871, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ccc54049-a714-4695-991e-a5a82834371d", + "resource": { + "resourceType": "Observation", + "id": "ccc54049-a714-4695-991e-a5a82834371d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 16.27, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", + "resource": { + "resourceType": "Observation", + "id": "8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 38.741, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", + "resource": { + "resourceType": "Observation", + "id": "fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 92.586, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", + "resource": { + "resourceType": "Observation", + "id": "1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 32.303, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:101e6eca-5997-42fd-b505-46a828396acc", + "resource": { + "resourceType": "Observation", + "id": "101e6eca-5997-42fd-b505-46a828396acc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 34.605, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d96c106b-54e1-45d4-887b-914e335e0150", + "resource": { + "resourceType": "Observation", + "id": "d96c106b-54e1-45d4-887b-914e335e0150", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 43.135, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", + "resource": { + "resourceType": "Observation", + "id": "9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 315, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", + "resource": { + "resourceType": "Observation", + "id": "0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 361.61, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59e54080-4536-4457-bdec-c8a220db1104", + "resource": { + "resourceType": "Observation", + "id": "59e54080-4536-4457-bdec-c8a220db1104", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 11.758, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff271586-f7a8-4b3d-9ab0-3a4a42ad56c2", + "resource": { + "resourceType": "Observation", + "id": "ff271586-f7a8-4b3d-9ab0-3a4a42ad56c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e114920-954b-4c05-972f-eafa39d65d74", + "resource": { + "resourceType": "Observation", + "id": "5e114920-954b-4c05-972f-eafa39d65d74", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.16, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59560246-0d32-426f-87f1-eb44a0ad9555", + "resource": { + "resourceType": "Procedure", + "id": "59560246-0d32-426f-87f1-eb44a0ad9555", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "performedPeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2165251f-c481-4462-8195-f3e298223329", + "resource": { + "resourceType": "MedicationRequest", + "id": "2165251f-c481-4462-8195-f3e298223329", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "authoredOn": "2010-12-05T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:bec69577-e95a-41b4-9398-7e4b2caabe24", + "resource": { + "resourceType": "Claim", + "id": "bec69577-e95a-41b4-9398-7e4b2caabe24", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "created": "2010-12-05T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:2165251f-c481-4462-8195-f3e298223329" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + } + ] + } + ], + "total": { + "value": 17.12, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:12b373cb-db80-4db3-9c55-770cc136b6e8", + "resource": { + "resourceType": "Immunization", + "id": "12b373cb-db80-4db3-9c55-770cc136b6e8", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "occurrenceDateTime": "2010-12-05T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c88c1240-10ed-47ac-9d38-50a26a1af4e5", + "resource": { + "resourceType": "Immunization", + "id": "c88c1240-10ed-47ac-9d38-50a26a1af4e5", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "occurrenceDateTime": "2010-12-05T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1b9f6afa-3e91-46ef-886b-2fbb0e2f7102", + "resource": { + "resourceType": "DiagnosticReport", + "id": "1b9f6afa-3e91-46ef-886b-2fbb0e2f7102", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:7be083f3-8f42-40a8-9c36-1ca04255875f", + "display": "Glucose" + }, + { + "reference": "urn:uuid:3777fdb4-4e26-441b-8760-59bf63aef0f0", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:a04df17d-795d-4971-b74d-605058b14505", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:29bc2948-0bed-40b6-9ac2-22e6129eef77", + "display": "Calcium" + }, + { + "reference": "urn:uuid:81f6ca85-5291-4753-8246-86bdfa0d8779", + "display": "Sodium" + }, + { + "reference": "urn:uuid:8e251d63-421d-406e-8cfd-3d6c98fea52d", + "display": "Potassium" + }, + { + "reference": "urn:uuid:8f778cd4-7df9-472d-870e-25920094f962", + "display": "Chloride" + }, + { + "reference": "urn:uuid:f7c5ce83-f870-42c0-9305-c627ef0a4128", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:71499e71-9aaf-4300-80e1-1d24d530136e", + "resource": { + "resourceType": "DiagnosticReport", + "id": "71499e71-9aaf-4300-80e1-1d24d530136e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:b24cd723-57ab-487c-aee1-6d613b11882b", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:410421b0-f849-4ef7-ae7f-66e1fb94e5b8", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:47c45751-fa57-45d4-a317-2ddbad218a1d", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:524e8995-4d9b-4def-b494-8e20df502c9b", + "resource": { + "resourceType": "DiagnosticReport", + "id": "524e8995-4d9b-4def-b494-8e20df502c9b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + }, + "effectiveDateTime": "2010-12-05T22:43:39-08:00", + "issued": "2010-12-05T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:aa4b8ede-2213-4a33-a72a-f474629cc496", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:ccc54049-a714-4695-991e-a5a82834371d", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:101e6eca-5997-42fd-b505-46a828396acc", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:d96c106b-54e1-45d4-887b-914e335e0150", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:59e54080-4536-4457-bdec-c8a220db1104", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e", + "resource": { + "resourceType": "Claim", + "id": "dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "created": "2010-12-05T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:12b373cb-db80-4db3-9c55-770cc136b6e8" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c88c1240-10ed-47ac-9d38-50a26a1af4e5" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:59560246-0d32-426f-87f1-eb44a0ad9555" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 4806.21, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:954efde3-c006-412e-9312-5beafd9bb414", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "954efde3-c006-412e-9312-5beafd9bb414", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2010-12-05T23:13:39-08:00", + "end": "2011-12-05T23:13:39-08:00" + }, + "created": "2010-12-05T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2010-12-05T22:43:39-08:00", + "end": "2010-12-05T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 4806.21, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 961.2420000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3844.9680000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 4806.21, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 4806.21, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 4069.8, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab", + "resource": { + "resourceType": "Encounter", + "id": "484f521b-b109-49a8-91c3-f11ccae0e8ab", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:52:39-07:00" + }, + "individual": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "display": "Dr. Dovie983 Mayert710" + } + } + ], + "period": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:52:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9ac27740-54c8-455f-b11b-0b0279c75fbd", + "resource": { + "resourceType": "Procedure", + "id": "9ac27740-54c8-455f-b11b-0b0279c75fbd", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68254000", + "display": "Removal of intrauterine device" + } + ], + "text": "Removal of intrauterine device" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" + }, + "performedPeriod": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:37:39-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5351d455-24d9-46d2-a7c0-2a30405f2181", + "resource": { + "resourceType": "Claim", + "id": "5351d455-24d9-46d2-a7c0-2a30405f2181", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:52:39-07:00" + }, + "created": "2011-06-27T00:52:39-07:00", + "provider": { + "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:9ac27740-54c8-455f-b11b-0b0279c75fbd" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + }, + "encounter": [ + { + "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68254000", + "display": "Removal of intrauterine device" + } + ], + "text": "Removal of intrauterine device" + }, + "net": { + "value": 9331.51, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c4c486df-6fa3-44d6-8235-5c4a9d8af510", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c4c486df-6fa3-44d6-8235-5c4a9d8af510", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "performer": [ + { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5351d455-24d9-46d2-a7c0-2a30405f2181" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2011-06-27T00:52:39-07:00", + "end": "2012-06-27T00:52:39-07:00" + }, + "created": "2011-06-27T00:52:39-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5351d455-24d9-46d2-a7c0-2a30405f2181" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + }, + "servicedPeriod": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:52:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68254000", + "display": "Removal of intrauterine device" + } + ], + "text": "Removal of intrauterine device" + }, + "servicedPeriod": { + "start": "2011-06-26T23:43:39-07:00", + "end": "2011-06-27T00:52:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 9331.51, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1866.3020000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7465.2080000000005, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 9331.51, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 9331.51, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7465.2080000000005, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0", + "resource": { + "resourceType": "Encounter", + "id": "c6beb7cf-35b9-415c-9dd0-130717faa8a0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1e88c727-6049-4a3f-89c1-e29d93447e25", + "resource": { + "resourceType": "Observation", + "id": "1e88c727-6049-4a3f-89c1-e29d93447e25", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 55.559, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5ced0f5-ec47-403d-b05c-76730f066dbf", + "resource": { + "resourceType": "Observation", + "id": "f5ced0f5-ec47-403d-b05c-76730f066dbf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f553a64-f551-42fc-9f41-daee856da6f5", + "resource": { + "resourceType": "Observation", + "id": "5f553a64-f551-42fc-9f41-daee856da6f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e9399377-66d1-4799-860d-f5a31f3ee5db", + "resource": { + "resourceType": "Observation", + "id": "e9399377-66d1-4799-860d-f5a31f3ee5db", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 73.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:79862766-83c1-43a4-98ac-03ee8a39aa39", + "resource": { + "resourceType": "Observation", + "id": "79862766-83c1-43a4-98ac-03ee8a39aa39", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 28.27, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4db6a529-675e-4a63-b965-53c9e8abfa8a", + "resource": { + "resourceType": "Observation", + "id": "4db6a529-675e-4a63-b965-53c9e8abfa8a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 117, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d340645d-91f9-43ad-961e-90a4d5413f0a", + "resource": { + "resourceType": "Observation", + "id": "d340645d-91f9-43ad-961e-90a4d5413f0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 92, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67a031c1-ffa4-4acc-8585-ad05de1f53df", + "resource": { + "resourceType": "Observation", + "id": "67a031c1-ffa4-4acc-8585-ad05de1f53df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:44547337-d65d-4939-b6a1-e50697a91474", + "resource": { + "resourceType": "Observation", + "id": "44547337-d65d-4939-b6a1-e50697a91474", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 84.64, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:621e86d2-a98c-40bf-adbe-8ff57d0004e8", + "resource": { + "resourceType": "Observation", + "id": "621e86d2-a98c-40bf-adbe-8ff57d0004e8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 17.21, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6caaac39-fa12-4e31-a101-c035df4d7943", + "resource": { + "resourceType": "Observation", + "id": "6caaac39-fa12-4e31-a101-c035df4d7943", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.77, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dfcbfbd2-13b5-41dc-82db-0b102079d64d", + "resource": { + "resourceType": "Observation", + "id": "dfcbfbd2-13b5-41dc-82db-0b102079d64d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.95, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f5446c7-f8a3-41db-a24c-1b76f535030a", + "resource": { + "resourceType": "Observation", + "id": "5f5446c7-f8a3-41db-a24c-1b76f535030a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 140.55, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d56c3b65-6525-4e02-9f95-19034fb17719", + "resource": { + "resourceType": "Observation", + "id": "d56c3b65-6525-4e02-9f95-19034fb17719", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 5, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:52ea8a78-ee19-4767-bded-ba61a20b704a", + "resource": { + "resourceType": "Observation", + "id": "52ea8a78-ee19-4767-bded-ba61a20b704a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 106.34, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d959c9b-95e5-4cc6-b50a-42faaa6e6308", + "resource": { + "resourceType": "Observation", + "id": "2d959c9b-95e5-4cc6-b50a-42faaa6e6308", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 21.61, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d44dbd7-d7b3-42d6-bdc8-419752261db5", + "resource": { + "resourceType": "Observation", + "id": "7d44dbd7-d7b3-42d6-bdc8-419752261db5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:80bb5b7a-6010-4f7a-9813-5eae64a461fa", + "resource": { + "resourceType": "Observation", + "id": "80bb5b7a-6010-4f7a-9813-5eae64a461fa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.36, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8b46f00-7b17-46a2-9d21-6d91f53dc8df", + "resource": { + "resourceType": "Procedure", + "id": "e8b46f00-7b17-46a2-9d21-6d91f53dc8df", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "performedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c3a06cc9-a555-4b36-b624-44aeb90b1ba3", + "resource": { + "resourceType": "MedicationRequest", + "id": "c3a06cc9-a555-4b36-b624-44aeb90b1ba3", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "authoredOn": "2011-12-11T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:c6b5665b-0ba0-458e-b8d3-095909e90b1d", + "resource": { + "resourceType": "Claim", + "id": "c6b5665b-0ba0-458e-b8d3-095909e90b1d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "created": "2011-12-11T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:c3a06cc9-a555-4b36-b624-44aeb90b1ba3" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + } + ] + } + ], + "total": { + "value": 23.82, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7baf31f8-b469-4a81-a8ff-a094c49df212", + "resource": { + "resourceType": "Immunization", + "id": "7baf31f8-b469-4a81-a8ff-a094c49df212", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "occurrenceDateTime": "2011-12-11T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8fa0ff7e-f7c2-4719-b9b6-620e15507668", + "resource": { + "resourceType": "Immunization", + "id": "8fa0ff7e-f7c2-4719-b9b6-620e15507668", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "occurrenceDateTime": "2011-12-11T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:6bf98825-4c89-42c7-bc08-671bce82c6b7", + "resource": { + "resourceType": "Immunization", + "id": "6bf98825-4c89-42c7-bc08-671bce82c6b7", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "occurrenceDateTime": "2011-12-11T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:855bbd86-3835-408d-92de-0ffa9986a8a2", + "resource": { + "resourceType": "DiagnosticReport", + "id": "855bbd86-3835-408d-92de-0ffa9986a8a2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + }, + "effectiveDateTime": "2011-12-11T22:43:39-08:00", + "issued": "2011-12-11T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:44547337-d65d-4939-b6a1-e50697a91474", + "display": "Glucose" + }, + { + "reference": "urn:uuid:621e86d2-a98c-40bf-adbe-8ff57d0004e8", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:6caaac39-fa12-4e31-a101-c035df4d7943", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:dfcbfbd2-13b5-41dc-82db-0b102079d64d", + "display": "Calcium" + }, + { + "reference": "urn:uuid:5f5446c7-f8a3-41db-a24c-1b76f535030a", + "display": "Sodium" + }, + { + "reference": "urn:uuid:d56c3b65-6525-4e02-9f95-19034fb17719", + "display": "Potassium" + }, + { + "reference": "urn:uuid:52ea8a78-ee19-4767-bded-ba61a20b704a", + "display": "Chloride" + }, + { + "reference": "urn:uuid:2d959c9b-95e5-4cc6-b50a-42faaa6e6308", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:aa77a4ce-50ba-4049-8866-f430d9c05879", + "resource": { + "resourceType": "Claim", + "id": "aa77a4ce-50ba-4049-8866-f430d9c05879", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "created": "2011-12-11T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7baf31f8-b469-4a81-a8ff-a094c49df212" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8fa0ff7e-f7c2-4719-b9b6-620e15507668" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:6bf98825-4c89-42c7-bc08-671bce82c6b7" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e8b46f00-7b17-46a2-9d21-6d91f53dc8df" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 5, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 7845.43, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:aa7595b7-f53d-4069-9382-5965eaafea81", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "aa7595b7-f53d-4069-9382-5965eaafea81", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "aa77a4ce-50ba-4049-8866-f430d9c05879" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2011-12-11T23:13:39-08:00", + "end": "2012-12-11T23:13:39-08:00" + }, + "created": "2011-12-11T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:aa77a4ce-50ba-4049-8866-f430d9c05879" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "servicedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2011-12-11T22:43:39-08:00", + "end": "2011-12-11T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 7845.43, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1569.0860000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 6276.344000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 7845.43, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 7845.43, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 6613.592000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955", + "resource": { + "resourceType": "Encounter", + "id": "fd5aa686-a8ac-4a82-bf0e-50974feec955", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-01-22T22:43:39-08:00", + "end": "2012-01-22T22:58:39-08:00" + }, + "individual": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "display": "Dr. Max124 Howell947" + } + } + ], + "period": { + "start": "2012-01-22T22:43:39-08:00", + "end": "2012-01-22T22:58:39-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23", + "resource": { + "resourceType": "Condition", + "id": "aa3202d2-c88e-4f71-a68c-d9ca53df9a23", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" + }, + "onsetDateTime": "2012-01-22T22:43:39-08:00", + "abatementDateTime": "2012-01-29T22:43:39-08:00", + "recordedDate": "2012-01-22T22:43:39-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:8aabc23e-3685-4406-92f2-8f94644170d6", + "resource": { + "resourceType": "Claim", + "id": "8aabc23e-3685-4406-92f2-8f94644170d6", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2012-01-22T22:43:39-08:00", + "end": "2012-01-22T22:58:39-08:00" + }, + "created": "2012-01-22T22:58:39-08:00", + "provider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b0ac1954-fcff-4e04-8c3c-ecc9d9cc3b79", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b0ac1954-fcff-4e04-8c3c-ecc9d9cc3b79", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "performer": [ + { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8aabc23e-3685-4406-92f2-8f94644170d6" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2012-01-22T22:58:39-08:00", + "end": "2013-01-22T22:58:39-08:00" + }, + "created": "2012-01-22T22:58:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8aabc23e-3685-4406-92f2-8f94644170d6" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2012-01-22T22:43:39-08:00", + "end": "2012-01-22T22:58:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "servicedPeriod": { + "start": "2012-01-22T22:43:39-08:00", + "end": "2012-01-22T22:58:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b", + "resource": { + "resourceType": "Encounter", + "id": "bd35fdbc-a30b-4a4d-aa39-41f7395d518b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2bb6c753-4248-464e-bde8-8029b4ffb551", + "resource": { + "resourceType": "Observation", + "id": "2bb6c753-4248-464e-bde8-8029b4ffb551", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 68.222, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8590cf5-2d61-4c67-b240-8073ced3df9d", + "resource": { + "resourceType": "Observation", + "id": "e8590cf5-2d61-4c67-b240-8073ced3df9d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2844371d-8829-4f08-bcdf-77b5807fcf02", + "resource": { + "resourceType": "Observation", + "id": "2844371d-8829-4f08-bcdf-77b5807fcf02", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33ab88c0-4e17-4689-b5f4-f9ef46c6520f", + "resource": { + "resourceType": "Observation", + "id": "33ab88c0-4e17-4689-b5f4-f9ef46c6520f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 75.2, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:659107a5-2495-4f01-906e-ba5b6d6c772e", + "resource": { + "resourceType": "Observation", + "id": "659107a5-2495-4f01-906e-ba5b6d6c772e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 29.01, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3cc5422-9dc6-422d-a89f-8d1888c06a59", + "resource": { + "resourceType": "Observation", + "id": "f3cc5422-9dc6-422d-a89f-8d1888c06a59", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 120, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6055ab5c-2274-4517-baaa-b2a643471122", + "resource": { + "resourceType": "Observation", + "id": "6055ab5c-2274-4517-baaa-b2a643471122", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 61, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:feb63539-8dfe-4d46-9075-f1bfb1ff28b3", + "resource": { + "resourceType": "Observation", + "id": "feb63539-8dfe-4d46-9075-f1bfb1ff28b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:046551b5-8be1-44ec-aeae-a01550e2a840", + "resource": { + "resourceType": "Observation", + "id": "046551b5-8be1-44ec-aeae-a01550e2a840", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 90.98, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:232470a3-b85c-4692-93d3-8ee2208dd6c7", + "resource": { + "resourceType": "Observation", + "id": "232470a3-b85c-4692-93d3-8ee2208dd6c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 18.04, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d3d23ee-4387-4281-80b5-ace382eda27f", + "resource": { + "resourceType": "Observation", + "id": "1d3d23ee-4387-4281-80b5-ace382eda27f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 1.22, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59364e97-3c77-4951-a226-5993ee64a447", + "resource": { + "resourceType": "Observation", + "id": "59364e97-3c77-4951-a226-5993ee64a447", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 8.74, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33129f88-4c75-4b90-a9a5-115d2c460712", + "resource": { + "resourceType": "Observation", + "id": "33129f88-4c75-4b90-a9a5-115d2c460712", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 140.33, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1bcc7db0-f025-4205-a6d9-625457a8a7c7", + "resource": { + "resourceType": "Observation", + "id": "1bcc7db0-f025-4205-a6d9-625457a8a7c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.11, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:04c373c5-2765-4e1c-99b4-690b02661135", + "resource": { + "resourceType": "Observation", + "id": "04c373c5-2765-4e1c-99b4-690b02661135", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 110.6, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfb00c59-e640-4531-9dff-b17f46fee964", + "resource": { + "resourceType": "Observation", + "id": "cfb00c59-e640-4531-9dff-b17f46fee964", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 22.67, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8ddfdc1c-ee9c-4887-b377-2f9049c273c8", + "resource": { + "resourceType": "Observation", + "id": "8ddfdc1c-ee9c-4887-b377-2f9049c273c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:81165715-4e9d-44e8-bad0-db827c8c09f6", + "resource": { + "resourceType": "Observation", + "id": "81165715-4e9d-44e8-bad0-db827c8c09f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.39, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d4a5c25-9381-4446-8338-54a129f342b4", + "resource": { + "resourceType": "Procedure", + "id": "0d4a5c25-9381-4446-8338-54a129f342b4", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "performedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ed7a4b76-8540-4854-bb88-5189a9946a9d", + "resource": { + "resourceType": "Procedure", + "id": "ed7a4b76-8540-4854-bb88-5189a9946a9d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "performedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T22:58:39-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:49b07d1a-14a4-4f59-a8f2-a2164030714c", + "resource": { + "resourceType": "MedicationRequest", + "id": "49b07d1a-14a4-4f59-a8f2-a2164030714c", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "authoredOn": "2012-12-16T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:552399db-be53-4538-a0cc-bae798c5b644", + "resource": { + "resourceType": "Claim", + "id": "552399db-be53-4538-a0cc-bae798c5b644", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "created": "2012-12-16T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:49b07d1a-14a4-4f59-a8f2-a2164030714c" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + } + ] + } + ], + "total": { + "value": 26.77, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:390d154c-e694-439f-982f-2f327c2bc201", + "resource": { + "resourceType": "Immunization", + "id": "390d154c-e694-439f-982f-2f327c2bc201", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "occurrenceDateTime": "2012-12-16T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:75939c84-7884-43d2-ac63-9b23551728d9", + "resource": { + "resourceType": "DiagnosticReport", + "id": "75939c84-7884-43d2-ac63-9b23551728d9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + }, + "effectiveDateTime": "2012-12-16T22:43:39-08:00", + "issued": "2012-12-16T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:046551b5-8be1-44ec-aeae-a01550e2a840", + "display": "Glucose" + }, + { + "reference": "urn:uuid:232470a3-b85c-4692-93d3-8ee2208dd6c7", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:1d3d23ee-4387-4281-80b5-ace382eda27f", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:59364e97-3c77-4951-a226-5993ee64a447", + "display": "Calcium" + }, + { + "reference": "urn:uuid:33129f88-4c75-4b90-a9a5-115d2c460712", + "display": "Sodium" + }, + { + "reference": "urn:uuid:1bcc7db0-f025-4205-a6d9-625457a8a7c7", + "display": "Potassium" + }, + { + "reference": "urn:uuid:04c373c5-2765-4e1c-99b4-690b02661135", + "display": "Chloride" + }, + { + "reference": "urn:uuid:cfb00c59-e640-4531-9dff-b17f46fee964", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:e9d6ffeb-ce12-4703-b888-380d272ffa78", + "resource": { + "resourceType": "Claim", + "id": "e9d6ffeb-ce12-4703-b888-380d272ffa78", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "created": "2012-12-16T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:390d154c-e694-439f-982f-2f327c2bc201" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:0d4a5c25-9381-4446-8338-54a129f342b4" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:ed7a4b76-8540-4854-bb88-5189a9946a9d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 7449.97, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 882.97, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8e38837b-8ebc-4ae2-b74a-f55f1bfa5671", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8e38837b-8ebc-4ae2-b74a-f55f1bfa5671", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e9d6ffeb-ce12-4703-b888-380d272ffa78" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2012-12-16T23:28:39-08:00", + "end": "2013-12-16T23:28:39-08:00" + }, + "created": "2012-12-16T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e9d6ffeb-ce12-4703-b888-380d272ffa78" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 7449.97, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1489.9940000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 5959.976000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 7449.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 7449.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2012-12-16T22:43:39-08:00", + "end": "2012-12-16T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 882.97, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 176.59400000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 706.3760000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 882.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 882.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 6778.768000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9", + "resource": { + "resourceType": "Encounter", + "id": "f52f6536-54a4-4df7-8148-fef06a43eab9", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3e858f82-b1aa-4fa0-941c-dcabeed081f1", + "resource": { + "resourceType": "Observation", + "id": "3e858f82-b1aa-4fa0-941c-dcabeed081f1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 60.593, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:736d1780-ebb7-4ee3-bd76-6e52f759f467", + "resource": { + "resourceType": "Observation", + "id": "736d1780-ebb7-4ee3-bd76-6e52f759f467", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:acbb6001-cb87-48d7-936d-b789e4c7a20f", + "resource": { + "resourceType": "Observation", + "id": "acbb6001-cb87-48d7-936d-b789e4c7a20f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf769eaa-0ca4-4706-abd5-52390613f5fc", + "resource": { + "resourceType": "Observation", + "id": "cf769eaa-0ca4-4706-abd5-52390613f5fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 77.1, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7739c6fb-6da8-464b-93af-f823c7d608f6", + "resource": { + "resourceType": "Observation", + "id": "7739c6fb-6da8-464b-93af-f823c7d608f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 29.76, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4219406-0e09-48b7-a7ab-912034ca8c4b", + "resource": { + "resourceType": "Observation", + "id": "d4219406-0e09-48b7-a7ab-912034ca8c4b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 135, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:52f95900-4e4b-4960-ae4b-4362373187da", + "resource": { + "resourceType": "Observation", + "id": "52f95900-4e4b-4960-ae4b-4362373187da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11796cbf-9dcd-427b-92f0-5f31d124d8a6", + "resource": { + "resourceType": "Observation", + "id": "11796cbf-9dcd-427b-92f0-5f31d124d8a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b08d9700-a684-48df-b6ea-844988b65172", + "resource": { + "resourceType": "Observation", + "id": "b08d9700-a684-48df-b6ea-844988b65172", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 85.01, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6aa2b320-bf22-4b4a-802f-90566174d079", + "resource": { + "resourceType": "Observation", + "id": "6aa2b320-bf22-4b4a-802f-90566174d079", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.45, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cbfc87c2-2d5e-4a0b-93dc-674d697ca838", + "resource": { + "resourceType": "Observation", + "id": "cbfc87c2-2d5e-4a0b-93dc-674d697ca838", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.84, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b23e8e1-0a91-4e70-a06f-7c145b564441", + "resource": { + "resourceType": "Observation", + "id": "8b23e8e1-0a91-4e70-a06f-7c145b564441", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 10.17, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3fc922ac-c871-4716-8378-ee6aebacc37d", + "resource": { + "resourceType": "Observation", + "id": "3fc922ac-c871-4716-8378-ee6aebacc37d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 139.93, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51c9eb84-cd7b-46d2-bf32-bf3b160ade97", + "resource": { + "resourceType": "Observation", + "id": "51c9eb84-cd7b-46d2-bf32-bf3b160ade97", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.31, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:314ab8a1-1b08-462d-a6e9-89972c40ecee", + "resource": { + "resourceType": "Observation", + "id": "314ab8a1-1b08-462d-a6e9-89972c40ecee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 108.41, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ad33c6dd-bd07-431d-9325-f3e2b8857951", + "resource": { + "resourceType": "Observation", + "id": "ad33c6dd-bd07-431d-9325-f3e2b8857951", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 20.44, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4d11c1f-62cb-49bf-9f37-6d20b08daced", + "resource": { + "resourceType": "Observation", + "id": "d4d11c1f-62cb-49bf-9f37-6d20b08daced", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 172.12, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddbbb510-58f8-4659-8248-8307e441e0cc", + "resource": { + "resourceType": "Observation", + "id": "ddbbb510-58f8-4659-8248-8307e441e0cc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 132.2, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ce68f389-c419-476e-9afc-6af644fa2950", + "resource": { + "resourceType": "Observation", + "id": "ce68f389-c419-476e-9afc-6af644fa2950", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 75, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:60f10017-43f7-47d5-a1d9-e77632adf3c0", + "resource": { + "resourceType": "Observation", + "id": "60f10017-43f7-47d5-a1d9-e77632adf3c0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 70.68, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:519a967f-38bd-4fe9-9790-4d2f05492a3e", + "resource": { + "resourceType": "Observation", + "id": "519a967f-38bd-4fe9-9790-4d2f05492a3e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d07d733-489d-40c6-a58f-304bcdc9290b", + "resource": { + "resourceType": "Observation", + "id": "2d07d733-489d-40c6-a58f-304bcdc9290b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.01, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:37d4b309-7901-4acf-9797-968f6bfcab14", + "resource": { + "resourceType": "Procedure", + "id": "37d4b309-7901-4acf-9797-968f6bfcab14", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "performedPeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e33496eb-6003-4555-8933-d39c10aa5bae", + "resource": { + "resourceType": "MedicationRequest", + "id": "e33496eb-6003-4555-8933-d39c10aa5bae", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "authoredOn": "2013-12-22T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:f9fc5cfd-6e2f-4332-a820-79b9199e6113", + "resource": { + "resourceType": "Claim", + "id": "f9fc5cfd-6e2f-4332-a820-79b9199e6113", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "created": "2013-12-22T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:e33496eb-6003-4555-8933-d39c10aa5bae" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + } + ] + } + ], + "total": { + "value": 24.06, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2", + "resource": { + "resourceType": "Immunization", + "id": "f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "occurrenceDateTime": "2013-12-22T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:dbcdc951-84cb-4870-a500-056c8a78aa6d", + "resource": { + "resourceType": "DiagnosticReport", + "id": "dbcdc951-84cb-4870-a500-056c8a78aa6d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:b08d9700-a684-48df-b6ea-844988b65172", + "display": "Glucose" + }, + { + "reference": "urn:uuid:6aa2b320-bf22-4b4a-802f-90566174d079", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:cbfc87c2-2d5e-4a0b-93dc-674d697ca838", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:8b23e8e1-0a91-4e70-a06f-7c145b564441", + "display": "Calcium" + }, + { + "reference": "urn:uuid:3fc922ac-c871-4716-8378-ee6aebacc37d", + "display": "Sodium" + }, + { + "reference": "urn:uuid:51c9eb84-cd7b-46d2-bf32-bf3b160ade97", + "display": "Potassium" + }, + { + "reference": "urn:uuid:314ab8a1-1b08-462d-a6e9-89972c40ecee", + "display": "Chloride" + }, + { + "reference": "urn:uuid:ad33c6dd-bd07-431d-9325-f3e2b8857951", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:4f2de63d-a4f1-4db1-a088-dc8f11fe2bc1", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4f2de63d-a4f1-4db1-a088-dc8f11fe2bc1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + }, + "effectiveDateTime": "2013-12-22T22:43:39-08:00", + "issued": "2013-12-22T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:d4d11c1f-62cb-49bf-9f37-6d20b08daced", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:ddbbb510-58f8-4659-8248-8307e441e0cc", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:ce68f389-c419-476e-9afc-6af644fa2950", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:60f10017-43f7-47d5-a1d9-e77632adf3c0", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d", + "resource": { + "resourceType": "Claim", + "id": "55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "created": "2013-12-22T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:37d4b309-7901-4acf-9797-968f6bfcab14" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 8942.99, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e8012c5a-73f8-482f-808b-92a098645d2a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e8012c5a-73f8-482f-808b-92a098645d2a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2013-12-22T23:13:39-08:00", + "end": "2014-12-22T23:13:39-08:00" + }, + "created": "2013-12-22T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2013-12-22T22:43:39-08:00", + "end": "2013-12-22T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 8942.99, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1788.598, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7154.392, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 8942.99, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 8942.99, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7266.808, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce", + "resource": { + "resourceType": "Encounter", + "id": "4d26768d-36e4-4eb5-8c91-73bcbdb251ce", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2d73a3c0-c78b-4456-a71d-443c2d29f9c3", + "resource": { + "resourceType": "Observation", + "id": "2d73a3c0-c78b-4456-a71d-443c2d29f9c3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 63.383, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a37713f0-c64c-476b-a543-0d4d9762a5b8", + "resource": { + "resourceType": "Observation", + "id": "a37713f0-c64c-476b-a543-0d4d9762a5b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b5eae6e-69bb-442f-908a-ffc6425516e5", + "resource": { + "resourceType": "Observation", + "id": "1b5eae6e-69bb-442f-908a-ffc6425516e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d24ff37-f10a-4906-a41a-1bbe779567e0", + "resource": { + "resourceType": "Observation", + "id": "0d24ff37-f10a-4906-a41a-1bbe779567e0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82f79397-68d4-4ee3-a60b-db191e4f1d35", + "resource": { + "resourceType": "Observation", + "id": "82f79397-68d4-4ee3-a60b-db191e4f1d35", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7cf4f1ee-59f4-446a-89bd-a7ad36fd4645", + "resource": { + "resourceType": "Observation", + "id": "7cf4f1ee-59f4-446a-89bd-a7ad36fd4645", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:928bbf24-634a-48c3-af65-dde6309b2ed9", + "resource": { + "resourceType": "Observation", + "id": "928bbf24-634a-48c3-af65-dde6309b2ed9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92049736-d9d4-4b69-bd9a-7e2f12239900", + "resource": { + "resourceType": "Observation", + "id": "92049736-d9d4-4b69-bd9a-7e2f12239900", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:89badd04-b9a1-4dfd-a31f-8f736a93402f", + "resource": { + "resourceType": "Observation", + "id": "89badd04-b9a1-4dfd-a31f-8f736a93402f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 85.07, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a875b76d-0043-4352-91f5-19ca9a8c3fcc", + "resource": { + "resourceType": "Observation", + "id": "a875b76d-0043-4352-91f5-19ca9a8c3fcc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 12.47, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:582b5b9b-e61d-41d6-958b-8d1af0e32a31", + "resource": { + "resourceType": "Observation", + "id": "582b5b9b-e61d-41d6-958b-8d1af0e32a31", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.8, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33572d1a-c496-4821-926f-037d901744f1", + "resource": { + "resourceType": "Observation", + "id": "33572d1a-c496-4821-926f-037d901744f1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.23, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", + "resource": { + "resourceType": "Observation", + "id": "68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 141.6, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:18a1e7e8-97ea-413a-afcd-71a9738dfc93", + "resource": { + "resourceType": "Observation", + "id": "18a1e7e8-97ea-413a-afcd-71a9738dfc93", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.39, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", + "resource": { + "resourceType": "Observation", + "id": "e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 102.89, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7911c2f1-afbe-4315-ba4b-b9b3e852027c", + "resource": { + "resourceType": "Observation", + "id": "7911c2f1-afbe-4315-ba4b-b9b3e852027c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 20.44, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d548fcc-9075-4a54-8ba8-d8c1f44fe56e", + "resource": { + "resourceType": "Observation", + "id": "3d548fcc-9075-4a54-8ba8-d8c1f44fe56e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9e294388-b1a3-4e2a-9f84-35355b7682db", + "resource": { + "resourceType": "Observation", + "id": "9e294388-b1a3-4e2a-9f84-35355b7682db", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.19, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c5c47bc9-3482-42eb-9aaf-1aeb90672c88", + "resource": { + "resourceType": "Procedure", + "id": "c5c47bc9-3482-42eb-9aaf-1aeb90672c88", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "performedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:35533f62-fa1a-4a62-bb02-a7179da222f0", + "resource": { + "resourceType": "Procedure", + "id": "35533f62-fa1a-4a62-bb02-a7179da222f0", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "performedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T22:58:39-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2674fcf5-9e55-4567-ba6a-54f9aeb4b88c", + "resource": { + "resourceType": "MedicationRequest", + "id": "2674fcf5-9e55-4567-ba6a-54f9aeb4b88c", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "authoredOn": "2014-12-28T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5244ddc1-58fe-4891-8e37-efca4e81e649", + "resource": { + "resourceType": "Claim", + "id": "5244ddc1-58fe-4891-8e37-efca4e81e649", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "created": "2014-12-28T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:2674fcf5-9e55-4567-ba6a-54f9aeb4b88c" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + } + ] + } + ], + "total": { + "value": 47.4, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:62640e6f-d0ed-4ddc-b104-9ada341ec832", + "resource": { + "resourceType": "Immunization", + "id": "62640e6f-d0ed-4ddc-b104-9ada341ec832", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "occurrenceDateTime": "2014-12-28T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7abeb6dc-e804-4498-988b-7e0b9035fdfc", + "resource": { + "resourceType": "DiagnosticReport", + "id": "7abeb6dc-e804-4498-988b-7e0b9035fdfc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + }, + "effectiveDateTime": "2014-12-28T22:43:39-08:00", + "issued": "2014-12-28T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:89badd04-b9a1-4dfd-a31f-8f736a93402f", + "display": "Glucose" + }, + { + "reference": "urn:uuid:a875b76d-0043-4352-91f5-19ca9a8c3fcc", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:582b5b9b-e61d-41d6-958b-8d1af0e32a31", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:33572d1a-c496-4821-926f-037d901744f1", + "display": "Calcium" + }, + { + "reference": "urn:uuid:68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", + "display": "Sodium" + }, + { + "reference": "urn:uuid:18a1e7e8-97ea-413a-afcd-71a9738dfc93", + "display": "Potassium" + }, + { + "reference": "urn:uuid:e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", + "display": "Chloride" + }, + { + "reference": "urn:uuid:7911c2f1-afbe-4315-ba4b-b9b3e852027c", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:89b59121-6403-4820-bd2d-86bae1fd8afe", + "resource": { + "resourceType": "Claim", + "id": "89b59121-6403-4820-bd2d-86bae1fd8afe", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "created": "2014-12-28T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:62640e6f-d0ed-4ddc-b104-9ada341ec832" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:c5c47bc9-3482-42eb-9aaf-1aeb90672c88" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:35533f62-fa1a-4a62-bb02-a7179da222f0" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 6270.22, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 360.08, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8f28d0f0-1855-4231-befc-33ac6bfde047", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8f28d0f0-1855-4231-befc-33ac6bfde047", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "89b59121-6403-4820-bd2d-86bae1fd8afe" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2014-12-28T23:28:39-08:00", + "end": "2015-12-28T23:28:39-08:00" + }, + "created": "2014-12-28T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:89b59121-6403-4820-bd2d-86bae1fd8afe" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 6270.22, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1254.044, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 5016.176, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 6270.22, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 6270.22, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2014-12-28T22:43:39-08:00", + "end": "2014-12-28T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 360.08, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 72.016, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 288.064, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 360.08, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 360.08, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 5416.656000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019", + "resource": { + "resourceType": "Encounter", + "id": "fabd54e8-f98b-43ab-8abf-c966701b6019", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:41:39-07:00" + }, + "individual": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "display": "Dr. Max124 Howell947" + } + } + ], + "period": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:41:39-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9067491c-015c-430b-a246-4c37fe2f8a09", + "resource": { + "resourceType": "Procedure", + "id": "9067491c-015c-430b-a246-4c37fe2f8a09", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" + }, + "performedPeriod": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:26:39-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3b2d05f9-7128-4b54-b24c-4220fef99a10", + "resource": { + "resourceType": "Claim", + "id": "3b2d05f9-7128-4b54-b24c-4220fef99a10", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:41:39-07:00" + }, + "created": "2015-10-03T00:41:39-07:00", + "provider": { + "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:9067491c-015c-430b-a246-4c37fe2f8a09" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 11770.61, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:01ba71a3-8baa-4589-ab87-23323a43584a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "01ba71a3-8baa-4589-ab87-23323a43584a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "performer": [ + { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3b2d05f9-7128-4b54-b24c-4220fef99a10" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2015-10-03T00:41:39-07:00", + "end": "2016-10-03T00:41:39-07:00" + }, + "created": "2015-10-03T00:41:39-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3b2d05f9-7128-4b54-b24c-4220fef99a10" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:41:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2015-10-02T23:43:39-07:00", + "end": "2015-10-03T00:41:39-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 11770.61, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2354.1220000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9416.488000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11770.61, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11770.61, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 9416.488000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e", + "resource": { + "resourceType": "Encounter", + "id": "8e3c7581-a34a-439c-8a9c-6bfea816ca4e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bfb3ffa2-d465-474d-b342-b15bd99c7d5a", + "resource": { + "resourceType": "Observation", + "id": "bfb3ffa2-d465-474d-b342-b15bd99c7d5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 71.731, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:60815011-e762-44a9-97bb-5522fb436590", + "resource": { + "resourceType": "Observation", + "id": "60815011-e762-44a9-97bb-5522fb436590", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:233ae8c8-7248-4730-9e87-125a986a581e", + "resource": { + "resourceType": "Observation", + "id": "233ae8c8-7248-4730-9e87-125a986a581e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:055fb244-41ee-4ba1-9257-136dadbb2ee4", + "resource": { + "resourceType": "Observation", + "id": "055fb244-41ee-4ba1-9257-136dadbb2ee4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6f9ee673-9a76-44bd-9611-abad79fa1744", + "resource": { + "resourceType": "Observation", + "id": "6f9ee673-9a76-44bd-9611-abad79fa1744", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:007d321f-df2f-4540-b8fc-fbf0e4fc812a", + "resource": { + "resourceType": "Observation", + "id": "007d321f-df2f-4540-b8fc-fbf0e4fc812a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 131, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e1528803-f67a-4609-8625-c1003d848f99", + "resource": { + "resourceType": "Observation", + "id": "e1528803-f67a-4609-8625-c1003d848f99", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 90, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac789242-35e2-47e0-b590-4ccc991397f9", + "resource": { + "resourceType": "Observation", + "id": "ac789242-35e2-47e0-b590-4ccc991397f9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:00915ad1-e8da-44b4-b216-93153231b84a", + "resource": { + "resourceType": "Observation", + "id": "00915ad1-e8da-44b4-b216-93153231b84a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 72.36, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ea0ab9c0-3864-4af7-9488-5750a22fe826", + "resource": { + "resourceType": "Observation", + "id": "ea0ab9c0-3864-4af7-9488-5750a22fe826", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 17.89, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5687f0d4-bbd0-402c-84af-f91d4cce566f", + "resource": { + "resourceType": "Observation", + "id": "5687f0d4-bbd0-402c-84af-f91d4cce566f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.86, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5f2abb5-10ad-4914-bb83-5f38c8e827f3", + "resource": { + "resourceType": "Observation", + "id": "f5f2abb5-10ad-4914-bb83-5f38c8e827f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.34, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f762dfda-ffbf-4d97-8ae9-564213b6acb6", + "resource": { + "resourceType": "Observation", + "id": "f762dfda-ffbf-4d97-8ae9-564213b6acb6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 138.48, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", + "resource": { + "resourceType": "Observation", + "id": "ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.08, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9b9877da-aad6-46b9-99e6-cc80aa86b1c8", + "resource": { + "resourceType": "Observation", + "id": "9b9877da-aad6-46b9-99e6-cc80aa86b1c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 105.9, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:493372ae-260b-412a-b3b7-461278f3cb77", + "resource": { + "resourceType": "Observation", + "id": "493372ae-260b-412a-b3b7-461278f3cb77", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 21.1, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a61b2a3-bd85-421c-96c2-c6eb382ea08e", + "resource": { + "resourceType": "Observation", + "id": "5a61b2a3-bd85-421c-96c2-c6eb382ea08e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 7.7907, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:874a2c15-1c59-44de-8821-96d9f1e0a8da", + "resource": { + "resourceType": "Observation", + "id": "874a2c15-1c59-44de-8821-96d9f1e0a8da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 3.9418, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df6499d7-8b55-43c4-b7df-efbdbcd67a09", + "resource": { + "resourceType": "Observation", + "id": "df6499d7-8b55-43c4-b7df-efbdbcd67a09", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 17.262, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d3a76fa9-be44-40d0-8ddc-a4b96a18a664", + "resource": { + "resourceType": "Observation", + "id": "d3a76fa9-be44-40d0-8ddc-a4b96a18a664", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 35.906, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5635958-2b38-4d0f-9de1-60956c7824dc", + "resource": { + "resourceType": "Observation", + "id": "f5635958-2b38-4d0f-9de1-60956c7824dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 93.807, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9553d8a-a97c-449d-a201-7a8bde168a89", + "resource": { + "resourceType": "Observation", + "id": "c9553d8a-a97c-449d-a201-7a8bde168a89", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.129, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d6768ab4-a243-40fb-b6aa-923ea3154778", + "resource": { + "resourceType": "Observation", + "id": "d6768ab4-a243-40fb-b6aa-923ea3154778", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 34.413, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3155135a-faa8-44be-abd2-56e1490dded8", + "resource": { + "resourceType": "Observation", + "id": "3155135a-faa8-44be-abd2-56e1490dded8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 45.889, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebd1df5e-a263-42be-bc0f-7e5e80807537", + "resource": { + "resourceType": "Observation", + "id": "ebd1df5e-a263-42be-bc0f-7e5e80807537", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 394.04, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", + "resource": { + "resourceType": "Observation", + "id": "fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 415.07, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bc6a414e-d54e-4553-a22b-726d4786977d", + "resource": { + "resourceType": "Observation", + "id": "bc6a414e-d54e-4553-a22b-726d4786977d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 10.033, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1073d9e6-4230-421f-95b2-d94e5cb6ba1b", + "resource": { + "resourceType": "Observation", + "id": "1073d9e6-4230-421f-95b2-d94e5cb6ba1b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:19c1b3c8-8e98-4c13-800d-31d529f7b5de", + "resource": { + "resourceType": "Observation", + "id": "19c1b3c8-8e98-4c13-800d-31d529f7b5de", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.22, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36497594-a72a-4914-977a-19c536b21d38", + "resource": { + "resourceType": "Procedure", + "id": "36497594-a72a-4914-977a-19c536b21d38", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "performedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7b813741-9057-40c5-b410-78bf1201a422", + "resource": { + "resourceType": "Procedure", + "id": "7b813741-9057-40c5-b410-78bf1201a422", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "performedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T22:58:39-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:08177039-c679-4659-b06f-cf18b1d5689f", + "resource": { + "resourceType": "MedicationRequest", + "id": "08177039-c679-4659-b06f-cf18b1d5689f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "authoredOn": "2016-01-03T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5ec76dfb-3152-4c4a-85ba-748687124fd8", + "resource": { + "resourceType": "Claim", + "id": "5ec76dfb-3152-4c4a-85ba-748687124fd8", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "created": "2016-01-03T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:08177039-c679-4659-b06f-cf18b1d5689f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + } + ] + } + ], + "total": { + "value": 20.82, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:172d8458-2b44-4b13-a99d-78e8ac972ae4", + "resource": { + "resourceType": "Immunization", + "id": "172d8458-2b44-4b13-a99d-78e8ac972ae4", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "occurrenceDateTime": "2016-01-03T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:38c19be3-9858-47af-a950-e6ad68dec814", + "resource": { + "resourceType": "DiagnosticReport", + "id": "38c19be3-9858-47af-a950-e6ad68dec814", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:00915ad1-e8da-44b4-b216-93153231b84a", + "display": "Glucose" + }, + { + "reference": "urn:uuid:ea0ab9c0-3864-4af7-9488-5750a22fe826", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:5687f0d4-bbd0-402c-84af-f91d4cce566f", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:f5f2abb5-10ad-4914-bb83-5f38c8e827f3", + "display": "Calcium" + }, + { + "reference": "urn:uuid:f762dfda-ffbf-4d97-8ae9-564213b6acb6", + "display": "Sodium" + }, + { + "reference": "urn:uuid:ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", + "display": "Potassium" + }, + { + "reference": "urn:uuid:9b9877da-aad6-46b9-99e6-cc80aa86b1c8", + "display": "Chloride" + }, + { + "reference": "urn:uuid:493372ae-260b-412a-b3b7-461278f3cb77", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f38942dd-07f2-407a-846e-980b4f723dce", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f38942dd-07f2-407a-846e-980b4f723dce", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + }, + "effectiveDateTime": "2016-01-03T22:43:39-08:00", + "issued": "2016-01-03T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:5a61b2a3-bd85-421c-96c2-c6eb382ea08e", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:874a2c15-1c59-44de-8821-96d9f1e0a8da", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:df6499d7-8b55-43c4-b7df-efbdbcd67a09", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:d3a76fa9-be44-40d0-8ddc-a4b96a18a664", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:f5635958-2b38-4d0f-9de1-60956c7824dc", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:c9553d8a-a97c-449d-a201-7a8bde168a89", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:d6768ab4-a243-40fb-b6aa-923ea3154778", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:3155135a-faa8-44be-abd2-56e1490dded8", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:ebd1df5e-a263-42be-bc0f-7e5e80807537", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:bc6a414e-d54e-4553-a22b-726d4786977d", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:5571a87e-7192-4b60-bf83-df6ab0fbefab", + "resource": { + "resourceType": "Claim", + "id": "5571a87e-7192-4b60-bf83-df6ab0fbefab", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "created": "2016-01-03T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:172d8458-2b44-4b13-a99d-78e8ac972ae4" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:36497594-a72a-4914-977a-19c536b21d38" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:7b813741-9057-40c5-b410-78bf1201a422" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 10455.00, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 920.83, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1d7e6885-3df9-46e7-a1cf-0c6241bfdfa8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1d7e6885-3df9-46e7-a1cf-0c6241bfdfa8", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5571a87e-7192-4b60-bf83-df6ab0fbefab" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2016-01-03T23:28:39-08:00", + "end": "2017-01-03T23:28:39-08:00" + }, + "created": "2016-01-03T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5571a87e-7192-4b60-bf83-df6ab0fbefab" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 10455.00, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2091.0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 8364.0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 10455.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 10455.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-03T22:43:39-08:00", + "end": "2016-01-03T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 920.83, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 184.16600000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 736.6640000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 920.83, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 920.83, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 9213.08, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978", + "resource": { + "resourceType": "Encounter", + "id": "e295fa5c-4876-4211-9f98-cb19ffad4978", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:135cea4d-a4fe-45c1-b7e1-d408e00f6122", + "resource": { + "resourceType": "Observation", + "id": "135cea4d-a4fe-45c1-b7e1-d408e00f6122", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 73.821, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33e2a4d1-07d4-4166-90e3-8cfaa28490b5", + "resource": { + "resourceType": "Observation", + "id": "33e2a4d1-07d4-4166-90e3-8cfaa28490b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84f60f1d-17ba-440d-8772-6ca72cc268e7", + "resource": { + "resourceType": "Observation", + "id": "84f60f1d-17ba-440d-8772-6ca72cc268e7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:638f670b-877d-4b93-9602-1bd211c059ac", + "resource": { + "resourceType": "Observation", + "id": "638f670b-877d-4b93-9602-1bd211c059ac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d04b82d-ce65-4b38-9a2e-4b2538082481", + "resource": { + "resourceType": "Observation", + "id": "2d04b82d-ce65-4b38-9a2e-4b2538082481", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c441282e-8448-4720-8a59-70253c394b77", + "resource": { + "resourceType": "Observation", + "id": "c441282e-8448-4720-8a59-70253c394b77", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 109, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e2509d7-4970-426d-8750-11e4c3383459", + "resource": { + "resourceType": "Observation", + "id": "6e2509d7-4970-426d-8750-11e4c3383459", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 76, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a95e98ac-6ed4-4951-b024-db57012b857d", + "resource": { + "resourceType": "Observation", + "id": "a95e98ac-6ed4-4951-b024-db57012b857d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a15d3f24-b640-4d9e-93de-86ef97606a73", + "resource": { + "resourceType": "Observation", + "id": "a15d3f24-b640-4d9e-93de-86ef97606a73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 69.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8d87cd49-2f03-46aa-852b-fa391aeef5ff", + "resource": { + "resourceType": "Observation", + "id": "8d87cd49-2f03-46aa-852b-fa391aeef5ff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 8.57, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:608a1fff-c93a-457e-b09c-3930d8a0dbc4", + "resource": { + "resourceType": "Observation", + "id": "608a1fff-c93a-457e-b09c-3930d8a0dbc4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac885c3a-9ac0-45e2-a9fa-88b683075c36", + "resource": { + "resourceType": "Observation", + "id": "ac885c3a-9ac0-45e2-a9fa-88b683075c36", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.27, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a435ae1-1fe9-4903-9631-a024c8234d17", + "resource": { + "resourceType": "Observation", + "id": "3a435ae1-1fe9-4903-9631-a024c8234d17", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 138.65, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e718f30a-6320-4c98-a49d-fdca25622adc", + "resource": { + "resourceType": "Observation", + "id": "e718f30a-6320-4c98-a49d-fdca25622adc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 3.96, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a8b3db26-2054-4703-9e9e-2f2095dc5316", + "resource": { + "resourceType": "Observation", + "id": "a8b3db26-2054-4703-9e9e-2f2095dc5316", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 106.11, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ea580ec-d735-479b-bfb9-85b64502d269", + "resource": { + "resourceType": "Observation", + "id": "2ea580ec-d735-479b-bfb9-85b64502d269", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 27.24, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", + "resource": { + "resourceType": "Observation", + "id": "6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 196.37, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9db44093-f9ca-40f7-b967-b222f6bba772", + "resource": { + "resourceType": "Observation", + "id": "9db44093-f9ca-40f7-b967-b222f6bba772", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 117.3, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:761a3bc8-ee20-4973-9139-dc7ef7141ea9", + "resource": { + "resourceType": "Observation", + "id": "761a3bc8-ee20-4973-9139-dc7ef7141ea9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 94.29, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3389b7f7-2a0c-4020-bb3d-161915a39a85", + "resource": { + "resourceType": "Observation", + "id": "3389b7f7-2a0c-4020-bb3d-161915a39a85", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.62, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:01c2b0d1-7b98-47c2-ae5d-db0e1ef31c18", + "resource": { + "resourceType": "Observation", + "id": "01c2b0d1-7b98-47c2-ae5d-db0e1ef31c18", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d31c6951-6ee5-4053-bb36-ee817b78a05a", + "resource": { + "resourceType": "Observation", + "id": "d31c6951-6ee5-4053-bb36-ee817b78a05a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "valueQuantity": { + "value": 5.9, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b1709e49-df90-48c5-ab67-caf89ada6aa2", + "resource": { + "resourceType": "Procedure", + "id": "b1709e49-df90-48c5-ab67-caf89ada6aa2", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "performedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:cd85bf62-5a17-44f2-ae66-44f7d7a5f379", + "resource": { + "resourceType": "Procedure", + "id": "cd85bf62-5a17-44f2-ae66-44f7d7a5f379", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "performedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T22:58:39-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4f446f5e-a358-449a-b1ae-f5a7b089f321", + "resource": { + "resourceType": "MedicationRequest", + "id": "4f446f5e-a358-449a-b1ae-f5a7b089f321", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "authoredOn": "2017-01-08T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:e888b203-22af-447e-9d74-f0893d521b83", + "resource": { + "resourceType": "Claim", + "id": "e888b203-22af-447e-9d74-f0893d521b83", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "created": "2017-01-08T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:4f446f5e-a358-449a-b1ae-f5a7b089f321" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + } + ] + } + ], + "total": { + "value": 10.09, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7041f895-b10b-47f4-992f-f46aa1e0ea67", + "resource": { + "resourceType": "Immunization", + "id": "7041f895-b10b-47f4-992f-f46aa1e0ea67", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "occurrenceDateTime": "2017-01-08T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a550bc16-c06b-4ba1-95ae-7fc33936ebf4", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a550bc16-c06b-4ba1-95ae-7fc33936ebf4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:a15d3f24-b640-4d9e-93de-86ef97606a73", + "display": "Glucose" + }, + { + "reference": "urn:uuid:8d87cd49-2f03-46aa-852b-fa391aeef5ff", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:608a1fff-c93a-457e-b09c-3930d8a0dbc4", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:ac885c3a-9ac0-45e2-a9fa-88b683075c36", + "display": "Calcium" + }, + { + "reference": "urn:uuid:3a435ae1-1fe9-4903-9631-a024c8234d17", + "display": "Sodium" + }, + { + "reference": "urn:uuid:e718f30a-6320-4c98-a49d-fdca25622adc", + "display": "Potassium" + }, + { + "reference": "urn:uuid:a8b3db26-2054-4703-9e9e-2f2095dc5316", + "display": "Chloride" + }, + { + "reference": "urn:uuid:2ea580ec-d735-479b-bfb9-85b64502d269", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7070af55-59a8-4f27-bacd-238f8f9cce37", + "resource": { + "resourceType": "DiagnosticReport", + "id": "7070af55-59a8-4f27-bacd-238f8f9cce37", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + }, + "effectiveDateTime": "2017-01-08T22:43:39-08:00", + "issued": "2017-01-08T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:9db44093-f9ca-40f7-b967-b222f6bba772", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:761a3bc8-ee20-4973-9139-dc7ef7141ea9", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:3389b7f7-2a0c-4020-bb3d-161915a39a85", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5", + "resource": { + "resourceType": "Claim", + "id": "43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "created": "2017-01-08T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7041f895-b10b-47f4-992f-f46aa1e0ea67" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b1709e49-df90-48c5-ab67-caf89ada6aa2" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:cd85bf62-5a17-44f2-ae66-44f7d7a5f379" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 13413.48, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 556.50, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:46e70939-620c-4560-9f97-3e55f463bf2b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "46e70939-620c-4560-9f97-3e55f463bf2b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2017-01-08T23:28:39-08:00", + "end": "2018-01-08T23:28:39-08:00" + }, + "created": "2017-01-08T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 13413.48, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2682.696, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 10730.784, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 13413.48, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 13413.48, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-08T22:43:39-08:00", + "end": "2017-01-08T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 556.50, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 111.30000000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 445.20000000000005, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 556.50, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 556.50, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 11288.4, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f", + "resource": { + "resourceType": "Encounter", + "id": "a798d69d-3188-4bdd-b0c7-22346acdf71f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b7ddb3cf-607f-4ab7-9f5a-d1180f3eef71", + "resource": { + "resourceType": "Observation", + "id": "b7ddb3cf-607f-4ab7-9f5a-d1180f3eef71", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 77.513, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d00b5f6-c3f8-471f-a9d4-7a4b04de4a66", + "resource": { + "resourceType": "Observation", + "id": "9d00b5f6-c3f8-471f-a9d4-7a4b04de4a66", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b240bf5c-67bb-42dd-8ca9-8734d02a3f55", + "resource": { + "resourceType": "Observation", + "id": "b240bf5c-67bb-42dd-8ca9-8734d02a3f55", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ea64c02-cafa-4e00-a0cc-ce889e8ba78b", + "resource": { + "resourceType": "Observation", + "id": "1ea64c02-cafa-4e00-a0cc-ce889e8ba78b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ab7b0cc-a56e-44fe-8589-c60f4e19b42d", + "resource": { + "resourceType": "Observation", + "id": "1ab7b0cc-a56e-44fe-8589-c60f4e19b42d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dec2c45a-3aaf-4189-a1f2-a22e79f671fc", + "resource": { + "resourceType": "Observation", + "id": "dec2c45a-3aaf-4189-a1f2-a22e79f671fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 130, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6339544d-a171-4a3d-a199-9707ab115717", + "resource": { + "resourceType": "Observation", + "id": "6339544d-a171-4a3d-a199-9707ab115717", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8ff42d2a-d512-4530-a8f2-c96c20dc306f", + "resource": { + "resourceType": "Observation", + "id": "8ff42d2a-d512-4530-a8f2-c96c20dc306f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3b617d2-944a-4e8d-9869-3796c1c9fd9a", + "resource": { + "resourceType": "Observation", + "id": "f3b617d2-944a-4e8d-9869-3796c1c9fd9a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 74.49, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aa78d1db-48e2-4fec-a765-02225e935783", + "resource": { + "resourceType": "Observation", + "id": "aa78d1db-48e2-4fec-a765-02225e935783", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 14.7, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", + "resource": { + "resourceType": "Observation", + "id": "a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.77, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:21fb774e-4614-46ed-b4a9-585f8c9f3116", + "resource": { + "resourceType": "Observation", + "id": "21fb774e-4614-46ed-b4a9-585f8c9f3116", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.62, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", + "resource": { + "resourceType": "Observation", + "id": "8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 143.53, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66005231-6d08-40fb-acc0-117a6333d101", + "resource": { + "resourceType": "Observation", + "id": "66005231-6d08-40fb-acc0-117a6333d101", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.86, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:487b2d25-a627-4de2-8dff-b73260d85de9", + "resource": { + "resourceType": "Observation", + "id": "487b2d25-a627-4de2-8dff-b73260d85de9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 110.58, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a9fffc1-fbae-4c6c-a9f8-1e459515239d", + "resource": { + "resourceType": "Observation", + "id": "8a9fffc1-fbae-4c6c-a9f8-1e459515239d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 25.74, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b27f8303-1b2c-4e9c-b23b-4fa4d723517c", + "resource": { + "resourceType": "Observation", + "id": "b27f8303-1b2c-4e9c-b23b-4fa4d723517c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:18141f56-2611-4b5e-b81a-3c59ad9995bd", + "resource": { + "resourceType": "Observation", + "id": "18141f56-2611-4b5e-b81a-3c59ad9995bd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "valueQuantity": { + "value": 5.8, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649", + "resource": { + "resourceType": "Procedure", + "id": "4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "performedPeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7a1740bb-9fed-48bf-8291-8034428337be", + "resource": { + "resourceType": "MedicationRequest", + "id": "7a1740bb-9fed-48bf-8291-8034428337be", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "authoredOn": "2018-01-14T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:521b3337-f168-4338-a7cc-97fedde9be21", + "resource": { + "resourceType": "Claim", + "id": "521b3337-f168-4338-a7cc-97fedde9be21", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "created": "2018-01-14T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:7a1740bb-9fed-48bf-8291-8034428337be" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + } + ] + } + ], + "total": { + "value": 35.14, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:523a74d7-7cb0-4d7e-8796-b6069eb4f254", + "resource": { + "resourceType": "Immunization", + "id": "523a74d7-7cb0-4d7e-8796-b6069eb4f254", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "occurrenceDateTime": "2018-01-14T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b2473f69-54db-4e35-80d5-e8dbf484f577", + "resource": { + "resourceType": "DiagnosticReport", + "id": "b2473f69-54db-4e35-80d5-e8dbf484f577", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + }, + "effectiveDateTime": "2018-01-14T22:43:39-08:00", + "issued": "2018-01-14T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:f3b617d2-944a-4e8d-9869-3796c1c9fd9a", + "display": "Glucose" + }, + { + "reference": "urn:uuid:aa78d1db-48e2-4fec-a765-02225e935783", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:21fb774e-4614-46ed-b4a9-585f8c9f3116", + "display": "Calcium" + }, + { + "reference": "urn:uuid:8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", + "display": "Sodium" + }, + { + "reference": "urn:uuid:66005231-6d08-40fb-acc0-117a6333d101", + "display": "Potassium" + }, + { + "reference": "urn:uuid:487b2d25-a627-4de2-8dff-b73260d85de9", + "display": "Chloride" + }, + { + "reference": "urn:uuid:8a9fffc1-fbae-4c6c-a9f8-1e459515239d", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:5807b9f9-621d-4c1b-81bc-539091b3ec86", + "resource": { + "resourceType": "Claim", + "id": "5807b9f9-621d-4c1b-81bc-539091b3ec86", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "created": "2018-01-14T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:523a74d7-7cb0-4d7e-8796-b6069eb4f254" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 9408.99, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:70312de5-f3e0-49f7-ab70-80f5554196d0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "70312de5-f3e0-49f7-ab70-80f5554196d0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5807b9f9-621d-4c1b-81bc-539091b3ec86" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2018-01-14T23:13:39-08:00", + "end": "2019-01-14T23:13:39-08:00" + }, + "created": "2018-01-14T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5807b9f9-621d-4c1b-81bc-539091b3ec86" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2018-01-14T22:43:39-08:00", + "end": "2018-01-14T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 9408.99, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1881.798, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7527.192, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 9408.99, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 9408.99, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7639.608, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2", + "resource": { + "resourceType": "Encounter", + "id": "64e01d2a-98d3-4934-9447-c27f361a15b2", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:26a97f52-9906-41b5-ac58-c30e636f82c0", + "resource": { + "resourceType": "Observation", + "id": "26a97f52-9906-41b5-ac58-c30e636f82c0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 54.986, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:edaf0e70-edc4-4f92-b3e1-e547ad28742b", + "resource": { + "resourceType": "Observation", + "id": "edaf0e70-edc4-4f92-b3e1-e547ad28742b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d60c87e8-182b-4ab1-bdeb-da169a5a5a76", + "resource": { + "resourceType": "Observation", + "id": "d60c87e8-182b-4ab1-bdeb-da169a5a5a76", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c970e1a-bb68-4c47-bc2d-28d0d0c8d938", + "resource": { + "resourceType": "Observation", + "id": "4c970e1a-bb68-4c47-bc2d-28d0d0c8d938", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:72dd89a6-49fd-46a7-b743-0c8f5f77b2ea", + "resource": { + "resourceType": "Observation", + "id": "72dd89a6-49fd-46a7-b743-0c8f5f77b2ea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b58f2406-ac64-4a5e-b703-b766fce60610", + "resource": { + "resourceType": "Observation", + "id": "b58f2406-ac64-4a5e-b703-b766fce60610", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 128, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:56946b5c-f75a-4a75-a5fb-daf51ad8a248", + "resource": { + "resourceType": "Observation", + "id": "56946b5c-f75a-4a75-a5fb-daf51ad8a248", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 98, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ad1591a1-778c-4eeb-b91b-93e546e6eb62", + "resource": { + "resourceType": "Observation", + "id": "ad1591a1-778c-4eeb-b91b-93e546e6eb62", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c8508776-0eec-4986-8d11-43ff06db4ec7", + "resource": { + "resourceType": "Observation", + "id": "c8508776-0eec-4986-8d11-43ff06db4ec7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 71.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", + "resource": { + "resourceType": "Observation", + "id": "0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 12.21, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30c82479-f6c5-4488-81be-cc60d82777b2", + "resource": { + "resourceType": "Observation", + "id": "30c82479-f6c5-4488-81be-cc60d82777b2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 1.33, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c71e1520-6255-4b1d-a642-67924c2e360b", + "resource": { + "resourceType": "Observation", + "id": "c71e1520-6255-4b1d-a642-67924c2e360b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.66, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47ef4e37-42fe-476f-b391-8b090bb5c98f", + "resource": { + "resourceType": "Observation", + "id": "47ef4e37-42fe-476f-b391-8b090bb5c98f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 140.36, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:835f2331-8c39-4c91-a24b-62932edd109b", + "resource": { + "resourceType": "Observation", + "id": "835f2331-8c39-4c91-a24b-62932edd109b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 4.49, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2004b881-cff4-4400-9767-ce1b4f1110ba", + "resource": { + "resourceType": "Observation", + "id": "2004b881-cff4-4400-9767-ce1b4f1110ba", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 103.84, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9904170d-b63f-48db-839b-63948bdab2f9", + "resource": { + "resourceType": "Observation", + "id": "9904170d-b63f-48db-839b-63948bdab2f9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 28.46, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0eeb5a4-8d9a-4855-934a-7273468a2eee", + "resource": { + "resourceType": "Observation", + "id": "b0eeb5a4-8d9a-4855-934a-7273468a2eee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a4ee721-c6c6-43e9-888e-f7ad49fc682d", + "resource": { + "resourceType": "Observation", + "id": "5a4ee721-c6c6-43e9-888e-f7ad49fc682d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "valueQuantity": { + "value": 5.81, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0a7424cc-ecae-4092-9620-2115372fb20b", + "resource": { + "resourceType": "Procedure", + "id": "0a7424cc-ecae-4092-9620-2115372fb20b", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "performedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b463ec82-b360-4639-b151-c3b8a369ab89", + "resource": { + "resourceType": "Procedure", + "id": "b463ec82-b360-4639-b151-c3b8a369ab89", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "performedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T22:58:39-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:cc39fe6e-11cd-483c-a66f-89ec15a72485", + "resource": { + "resourceType": "MedicationRequest", + "id": "cc39fe6e-11cd-483c-a66f-89ec15a72485", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "authoredOn": "2019-01-20T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:31313185-8995-4f14-9b0d-811fb8d9020a", + "resource": { + "resourceType": "Claim", + "id": "31313185-8995-4f14-9b0d-811fb8d9020a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "created": "2019-01-20T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:cc39fe6e-11cd-483c-a66f-89ec15a72485" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + } + ] + } + ], + "total": { + "value": 34.72, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:54e00fdd-e368-4c6e-9335-b1001aa3ce96", + "resource": { + "resourceType": "Immunization", + "id": "54e00fdd-e368-4c6e-9335-b1001aa3ce96", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "occurrenceDateTime": "2019-01-20T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a635116e-7100-454a-aacf-7ecfee9f5e47", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a635116e-7100-454a-aacf-7ecfee9f5e47", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + }, + "effectiveDateTime": "2019-01-20T22:43:39-08:00", + "issued": "2019-01-20T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:c8508776-0eec-4986-8d11-43ff06db4ec7", + "display": "Glucose" + }, + { + "reference": "urn:uuid:0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:30c82479-f6c5-4488-81be-cc60d82777b2", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:c71e1520-6255-4b1d-a642-67924c2e360b", + "display": "Calcium" + }, + { + "reference": "urn:uuid:47ef4e37-42fe-476f-b391-8b090bb5c98f", + "display": "Sodium" + }, + { + "reference": "urn:uuid:835f2331-8c39-4c91-a24b-62932edd109b", + "display": "Potassium" + }, + { + "reference": "urn:uuid:2004b881-cff4-4400-9767-ce1b4f1110ba", + "display": "Chloride" + }, + { + "reference": "urn:uuid:9904170d-b63f-48db-839b-63948bdab2f9", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:8c44ac1b-9c63-485b-b706-da3bf7b05ce7", + "resource": { + "resourceType": "Claim", + "id": "8c44ac1b-9c63-485b-b706-da3bf7b05ce7", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "created": "2019-01-20T23:28:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:54e00fdd-e368-4c6e-9335-b1001aa3ce96" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:0a7424cc-ecae-4092-9620-2115372fb20b" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:b463ec82-b360-4639-b151-c3b8a369ab89" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 7705.04, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 553.63, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:03b8f82d-9d61-4703-87f3-42089f7b0444", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "03b8f82d-9d61-4703-87f3-42089f7b0444", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8c44ac1b-9c63-485b-b706-da3bf7b05ce7" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2019-01-20T23:28:39-08:00", + "end": "2020-01-20T23:28:39-08:00" + }, + "created": "2019-01-20T23:28:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8c44ac1b-9c63-485b-b706-da3bf7b05ce7" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 7705.04, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1541.008, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 6164.032, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 7705.04, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 7705.04, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2019-01-20T22:43:39-08:00", + "end": "2019-01-20T23:28:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 553.63, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 110.726, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 442.904, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 553.63, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 553.63, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 6719.352000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0", + "resource": { + "resourceType": "Encounter", + "id": "fc0f7e2d-938d-4f37-8f1f-726e614bb0d0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Mrs. Lorette239 Marvin195" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "individual": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + } + } + ], + "period": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4ed1bb6d-ea9d-43da-92d4-f881780ba67c", + "resource": { + "resourceType": "Observation", + "id": "4ed1bb6d-ea9d-43da-92d4-f881780ba67c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "19926-5", + "display": "FEV1/FVC" + } + ], + "text": "FEV1/FVC" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 74.344, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6abfde09-c630-43b7-8e14-1573dd652ae7", + "resource": { + "resourceType": "Observation", + "id": "6abfde09-c630-43b7-8e14-1573dd652ae7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 161, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bd5fa694-5a73-4cc2-8c98-ef7573efe2dc", + "resource": { + "resourceType": "Observation", + "id": "bd5fa694-5a73-4cc2-8c98-ef7573efe2dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c3f55167-2ae7-412e-b9d3-b713903b22d9", + "resource": { + "resourceType": "Observation", + "id": "c3f55167-2ae7-412e-b9d3-b713903b22d9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:09c2a3f5-cd18-4bf1-9e17-f26dff936459", + "resource": { + "resourceType": "Observation", + "id": "09c2a3f5-cd18-4bf1-9e17-f26dff936459", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 30.4, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1af95c19-cd46-4bfc-9150-b417cd8a05de", + "resource": { + "resourceType": "Observation", + "id": "1af95c19-cd46-4bfc-9150-b417cd8a05de", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 139, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c7d87b7-088a-4fba-8c26-7255530ceabd", + "resource": { + "resourceType": "Observation", + "id": "3c7d87b7-088a-4fba-8c26-7255530ceabd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 86, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac1f5163-1e12-4dce-97cd-8dc303617416", + "resource": { + "resourceType": "Observation", + "id": "ac1f5163-1e12-4dce-97cd-8dc303617416", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d753d661-2633-4635-b0d2-ef8fdcbcfdfc", + "resource": { + "resourceType": "Observation", + "id": "d753d661-2633-4635-b0d2-ef8fdcbcfdfc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 74.49, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f21625c-79c1-466c-af0c-1d2ee5e1ff43", + "resource": { + "resourceType": "Observation", + "id": "5f21625c-79c1-466c-af0c-1d2ee5e1ff43", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 16.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7542f0dd-48c7-4d55-aaee-af92a2a21a5e", + "resource": { + "resourceType": "Observation", + "id": "7542f0dd-48c7-4d55-aaee-af92a2a21a5e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 0.96, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d95b95e9-82c7-4384-8dfc-89983af44610", + "resource": { + "resourceType": "Observation", + "id": "d95b95e9-82c7-4384-8dfc-89983af44610", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 9.12, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", + "resource": { + "resourceType": "Observation", + "id": "86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 141.99, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6866baa3-d56e-4e8a-b32f-a47b505acfd3", + "resource": { + "resourceType": "Observation", + "id": "6866baa3-d56e-4e8a-b32f-a47b505acfd3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 3.92, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:795f464e-3921-4b38-a87d-72529e633ee2", + "resource": { + "resourceType": "Observation", + "id": "795f464e-3921-4b38-a87d-72529e633ee2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 103.19, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51bb4bb3-94bb-408c-89d5-e470b389dbc8", + "resource": { + "resourceType": "Observation", + "id": "51bb4bb3-94bb-408c-89d5-e470b389dbc8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 27.47, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:af1547cb-5908-4d07-bac1-0bf1eb9d295d", + "resource": { + "resourceType": "Observation", + "id": "af1547cb-5908-4d07-bac1-0bf1eb9d295d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 189.91, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76ba1a3f-068b-491b-82a4-21f5e4c30e74", + "resource": { + "resourceType": "Observation", + "id": "76ba1a3f-068b-491b-82a4-21f5e4c30e74", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 116.92, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9c550c5-54da-459e-bb00-81a27fc89390", + "resource": { + "resourceType": "Observation", + "id": "a9c550c5-54da-459e-bb00-81a27fc89390", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 87.95, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f96acc88-5535-41be-a4c3-17a762512090", + "resource": { + "resourceType": "Observation", + "id": "f96acc88-5535-41be-a4c3-17a762512090", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 78.57, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82b37be7-5742-4ec7-9ba2-e3b5b8e4930f", + "resource": { + "resourceType": "Observation", + "id": "82b37be7-5742-4ec7-9ba2-e3b5b8e4930f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "8517006", + "display": "Former smoker" + } + ], + "text": "Former smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd2afe05-4b40-46f6-99f1-de9b9d836621", + "resource": { + "resourceType": "Observation", + "id": "dd2afe05-4b40-46f6-99f1-de9b9d836621", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "valueQuantity": { + "value": 6.19, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d479c20-61eb-4cf2-b3cb-a03aa7a276fb", + "resource": { + "resourceType": "Procedure", + "id": "7d479c20-61eb-4cf2-b3cb-a03aa7a276fb", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "performedPeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T22:58:39-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", + "display": "Pulmonary emphysema (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf", + "resource": { + "resourceType": "MedicationRequest", + "id": "4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "896209", + "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + } + ], + "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "authoredOn": "2020-01-26T22:43:39-08:00", + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "display": "Dr. Cheryle584 Koelpin146" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:970138e8-c02f-4600-bd18-60b60d9c837f", + "resource": { + "resourceType": "Claim", + "id": "970138e8-c02f-4600-bd18-60b60d9c837f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "created": "2020-01-26T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + } + ] + } + ], + "total": { + "value": 17.59, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:140cb569-a32b-4b6b-abc3-85157b9ee101", + "resource": { + "resourceType": "Immunization", + "id": "140cb569-a32b-4b6b-abc3-85157b9ee101", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "occurrenceDateTime": "2020-01-26T22:43:39-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c1dcfdd5-7c2d-4654-94f2-9eaaa22137a7", + "resource": { + "resourceType": "DiagnosticReport", + "id": "c1dcfdd5-7c2d-4654-94f2-9eaaa22137a7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:d753d661-2633-4635-b0d2-ef8fdcbcfdfc", + "display": "Glucose" + }, + { + "reference": "urn:uuid:5f21625c-79c1-466c-af0c-1d2ee5e1ff43", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:7542f0dd-48c7-4d55-aaee-af92a2a21a5e", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:d95b95e9-82c7-4384-8dfc-89983af44610", + "display": "Calcium" + }, + { + "reference": "urn:uuid:86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", + "display": "Sodium" + }, + { + "reference": "urn:uuid:6866baa3-d56e-4e8a-b32f-a47b505acfd3", + "display": "Potassium" + }, + { + "reference": "urn:uuid:795f464e-3921-4b38-a87d-72529e633ee2", + "display": "Chloride" + }, + { + "reference": "urn:uuid:51bb4bb3-94bb-408c-89d5-e470b389dbc8", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f1737ab8-8d19-436f-aea9-a010acce57da", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f1737ab8-8d19-436f-aea9-a010acce57da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "encounter": { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + }, + "effectiveDateTime": "2020-01-26T22:43:39-08:00", + "issued": "2020-01-26T22:43:39.914-08:00", + "result": [ + { + "reference": "urn:uuid:af1547cb-5908-4d07-bac1-0bf1eb9d295d", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:76ba1a3f-068b-491b-82a4-21f5e4c30e74", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:a9c550c5-54da-459e-bb00-81a27fc89390", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:f96acc88-5535-41be-a4c3-17a762512090", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:40556fd6-be95-44a1-beeb-bed2a8cb1513", + "resource": { + "resourceType": "Claim", + "id": "40556fd6-be95-44a1-beeb-bed2a8cb1513", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", + "display": "Lorette239 Marvin195" + }, + "billablePeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "created": "2020-01-26T23:13:39-08:00", + "provider": { + "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:140cb569-a32b-4b6b-abc3-85157b9ee101" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7d479c20-61eb-4cf2-b3cb-a03aa7a276fb" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "net": { + "value": 14224.84, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8f1b7b95-c113-48fc-a6a4-b73187564ffe", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8f1b7b95-c113-48fc-a6a4-b73187564ffe", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "requester": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "performer": [ + { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "40556fd6-be95-44a1-beeb-bed2a8cb1513" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" + }, + "billablePeriod": { + "start": "2020-01-26T23:13:39-08:00", + "end": "2021-01-26T23:13:39-08:00" + }, + "created": "2020-01-26T23:13:39-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:40556fd6-be95-44a1-beeb-bed2a8cb1513" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "127783003", + "display": "Spirometry (procedure)" + } + ], + "text": "Spirometry (procedure)" + }, + "servicedPeriod": { + "start": "2020-01-26T22:43:39-08:00", + "end": "2020-01-26T23:13:39-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 14224.84, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2844.9680000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 11379.872000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 14224.84, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 14224.84, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 11492.288, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json new file mode 100644 index 000000000000..a3a593a48dea --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json @@ -0,0 +1,16971 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "resource": { + "resourceType": "Patient", + "id": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7883787131534806314 Population seed: 1586368870505
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Cinthia596 Thompson596" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", + "valueAddress": { + "city": "Quincy", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.0 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 16.0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "SS", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-52-7917" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99926136" + } + ], + "name": [ + { + "use": "official", + "family": "Cole117", + "given": [ + "Norberto865" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-257-6498", + "use": "home" + } + ], + "gender": "male", + "birthDate": "2003-12-29", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 41.64238723329318 + }, + { + "url": "longitude", + "valueDecimal": -71.16534881278795 + } + ] + } + ], + "line": [ + "436 Rogahn Alley" + ], + "city": "Fall River", + "state": "Massachusetts", + "postalCode": "02747", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "resource": { + "resourceType": "Organization", + "id": "75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "75bdb2ec-cbd0-35af-b0fe-18b111890b67" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP235727", + "address": [ + { + "line": [ + "67 SLADES FERRY BLVD" + ], + "city": "SOMERSET", + "state": "MA", + "postalCode": "02726", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "resource": { + "resourceType": "Practitioner", + "id": "b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999931859" + } + ], + "active": true, + "name": [ + { + "family": "Mante251", + "given": [ + "Bertie593" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Bertie593.Mante251@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "67 SLADES FERRY BLVD" + ], + "city": "SOMERSET", + "state": "MA", + "postalCode": "02726", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86", + "resource": { + "resourceType": "Encounter", + "id": "3c8db206-734b-4a90-a8bd-c091a117dc86", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2010-12-20T08:41:37-08:00", + "end": "2010-12-20T08:56:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2010-12-20T08:41:37-08:00", + "end": "2010-12-20T08:56:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:23a2b208-2135-4a6c-885c-e246549a3f8d", + "resource": { + "resourceType": "Observation", + "id": "23a2b208-2135-4a6c-885c-e246549a3f8d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 119.1, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e88cac97-70b3-4ec2-bcf4-a1c493d4cdc3", + "resource": { + "resourceType": "Observation", + "id": "e88cac97-70b3-4ec2-bcf4-a1c493d4cdc3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3125fbf8-307d-43a9-bb38-409595e448cc", + "resource": { + "resourceType": "Observation", + "id": "3125fbf8-307d-43a9-bb38-409595e448cc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 26.9, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35d44b26-697d-4eb0-b661-44a50a369a16", + "resource": { + "resourceType": "Observation", + "id": "35d44b26-697d-4eb0-b661-44a50a369a16", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 18.96, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67c2496e-7394-40c1-ae13-a7614305fe0a", + "resource": { + "resourceType": "Observation", + "id": "67c2496e-7394-40c1-ae13-a7614305fe0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 94.625, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92be6692-adf0-49ea-abf1-447192a3333d", + "resource": { + "resourceType": "Observation", + "id": "92be6692-adf0-49ea-abf1-447192a3333d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e5638cb4-8a26-4446-b6e5-100fe98d5166", + "resource": { + "resourceType": "Observation", + "id": "e5638cb4-8a26-4446-b6e5-100fe98d5166", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 73, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ec688fc5-cbf0-43cd-a7e4-86be00621b30", + "resource": { + "resourceType": "Observation", + "id": "ec688fc5-cbf0-43cd-a7e4-86be00621b30", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:097893e3-f0e0-4cb9-ac46-18f37b9dc94a", + "resource": { + "resourceType": "Observation", + "id": "097893e3-f0e0-4cb9-ac46-18f37b9dc94a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "effectiveDateTime": "2010-12-20T08:41:37-08:00", + "issued": "2010-12-20T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4", + "resource": { + "resourceType": "Immunization", + "id": "1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + }, + "occurrenceDateTime": "2010-12-20T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4b322488-0670-4432-8115-e4853cda7d23", + "resource": { + "resourceType": "Claim", + "id": "4b322488-0670-4432-8115-e4853cda7d23", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2010-12-20T08:41:37-08:00", + "end": "2010-12-20T08:56:37-08:00" + }, + "created": "2010-12-20T08:56:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:238fb15a-a86f-40a9-acb8-b9b801234ac2", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "238fb15a-a86f-40a9-acb8-b9b801234ac2", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4b322488-0670-4432-8115-e4853cda7d23" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2010-12-20T08:56:37-08:00", + "end": "2011-12-20T08:56:37-08:00" + }, + "created": "2010-12-20T08:56:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4b322488-0670-4432-8115-e4853cda7d23" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2010-12-20T08:41:37-08:00", + "end": "2010-12-20T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2010-12-20T08:41:37-08:00", + "end": "2010-12-20T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b", + "resource": { + "resourceType": "Encounter", + "id": "77756f23-e4c6-4726-876a-df37611c202b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0902d9d3-1ae2-4240-b2ba-9df1e1e31b38", + "resource": { + "resourceType": "Observation", + "id": "0902d9d3-1ae2-4240-b2ba-9df1e1e31b38", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 125.1, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4301ad0f-7cfc-4a73-8445-607a0b01388a", + "resource": { + "resourceType": "Observation", + "id": "4301ad0f-7cfc-4a73-8445-607a0b01388a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51d6cc0f-ffd3-44f7-b4ac-0a17a252dfdf", + "resource": { + "resourceType": "Observation", + "id": "51d6cc0f-ffd3-44f7-b4ac-0a17a252dfdf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 30.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3792f8fc-4c29-4f60-bceb-f0940eb4be32", + "resource": { + "resourceType": "Observation", + "id": "3792f8fc-4c29-4f60-bceb-f0940eb4be32", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 19.51, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7b707ea1-3ac8-427a-80d7-d21da0351a40", + "resource": { + "resourceType": "Observation", + "id": "7b707ea1-3ac8-427a-80d7-d21da0351a40", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 93.686, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3208461d-688e-471d-90b7-d456109d073f", + "resource": { + "resourceType": "Observation", + "id": "3208461d-688e-471d-90b7-d456109d073f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 124, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7973049-699a-413c-8f00-b52fc1ecac6e", + "resource": { + "resourceType": "Observation", + "id": "b7973049-699a-413c-8f00-b52fc1ecac6e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 90, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d8bd8eb-4404-460a-8f37-d9016facedec", + "resource": { + "resourceType": "Observation", + "id": "7d8bd8eb-4404-460a-8f37-d9016facedec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1ccae92-2e8d-4ca4-a47e-53e8a684de26", + "resource": { + "resourceType": "Observation", + "id": "f1ccae92-2e8d-4ca4-a47e-53e8a684de26", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "effectiveDateTime": "2011-12-26T08:41:37-08:00", + "issued": "2011-12-26T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6bb3e397-48b3-4830-94e8-626289c8279e", + "resource": { + "resourceType": "Procedure", + "id": "6bb3e397-48b3-4830-94e8-626289c8279e", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "performedPeriod": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T08:56:37-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4a67f3d4-0e39-4a6c-996e-2235dbdc6674", + "resource": { + "resourceType": "Immunization", + "id": "4a67f3d4-0e39-4a6c-996e-2235dbdc6674", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + }, + "occurrenceDateTime": "2011-12-26T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54", + "resource": { + "resourceType": "Claim", + "id": "4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "created": "2011-12-26T09:11:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4a67f3d4-0e39-4a6c-996e-2235dbdc6674" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:6bb3e397-48b3-4830-94e8-626289c8279e" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 970.60, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ca620d16-0e4d-4829-8726-48661c7b88d6", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ca620d16-0e4d-4829-8726-48661c7b88d6", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2011-12-26T09:11:37-08:00", + "end": "2012-12-26T09:11:37-08:00" + }, + "created": "2011-12-26T09:11:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2011-12-26T08:41:37-08:00", + "end": "2011-12-26T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 970.60, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 194.12, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 776.48, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 970.60, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 970.60, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 888.8960000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "resource": { + "resourceType": "Organization", + "id": "23834663-ed53-3da9-b330-d6e1ecb8428e", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "23834663-ed53-3da9-b330-d6e1ecb8428e" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SOUTHCOAST HOSPITAL GROUP, INC", + "telecom": [ + { + "system": "phone", + "value": "5086793131" + } + ], + "address": [ + { + "line": [ + "363 HIGHLAND AVENUE" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02720", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "resource": { + "resourceType": "Practitioner", + "id": "f411d1de-1d0c-365f-8a71-8d225cb962af", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999699" + } + ], + "active": true, + "name": [ + { + "family": "Brakus656", + "given": [ + "Gillian484" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Gillian484.Brakus656@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "363 HIGHLAND AVENUE" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02720", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788", + "resource": { + "resourceType": "Encounter", + "id": "93bc0473-3b31-4a1e-8456-382950f8b788", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "individual": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + } + ], + "period": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe", + "resource": { + "resourceType": "Condition", + "id": "fabcf585-727e-4e48-bcac-2f51c1588ebe", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + }, + "onsetDateTime": "2012-05-15T09:41:37-07:00", + "abatementDateTime": "2012-06-19T09:41:37-07:00", + "recordedDate": "2012-05-15T09:41:37-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:1123b665-2688-4488-bd77-96f11c96ba58", + "resource": { + "resourceType": "MedicationRequest", + "id": "1123b665-2688-4488-bd77-96f11c96ba58", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + }, + "authoredOn": "2012-05-15T09:41:37-07:00", + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:457d2594-ad89-4c5a-a863-eb983d000719", + "resource": { + "resourceType": "Claim", + "id": "457d2594-ad89-4c5a-a863-eb983d000719", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "created": "2012-05-15T10:41:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:1123b665-2688-4488-bd77-96f11c96ba58" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + } + ] + } + ], + "total": { + "value": 4.14, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba", + "resource": { + "resourceType": "CareTeam", + "id": "d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba", + "status": "inactive", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + }, + "period": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-06-19T09:41:37-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:50175367-bf28-4bc0-87d6-88f8dc0f984e", + "resource": { + "resourceType": "CarePlan", + "id": "50175367-bf28-4bc0-87d6-88f8dc0f984e", + "text": { + "status": "generated", + "div": "
    Care Plan for Physical therapy procedure.
    Activities:
    • Physical therapy procedure
    • Physical therapy procedure

    Care plan is meant to treat Sprain of ankle.
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "91251008", + "display": "Physical therapy procedure" + } + ], + "text": "Physical therapy procedure" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + }, + "period": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-06-19T09:41:37-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba" + } + ], + "addresses": [ + { + "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229586001", + "display": "Rest, ice, compression and elevation treatment programme" + } + ], + "text": "Rest, ice, compression and elevation treatment programme" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229070002", + "display": "Stretching exercises" + } + ], + "text": "Stretching exercises" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:f285810f-2b70-4abb-afdd-db15790f17f1", + "resource": { + "resourceType": "Claim", + "id": "f285810f-2b70-4abb-afdd-db15790f17f1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "created": "2012-05-15T10:41:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1c10ff57-e9fa-4c09-838c-05b6854df0f8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1c10ff57-e9fa-4c09-838c-05b6854df0f8", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "performer": [ + { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f285810f-2b70-4abb-afdd-db15790f17f1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-05-15T10:41:37-07:00", + "end": "2013-05-15T10:41:37-07:00" + }, + "created": "2012-05-15T10:41:37-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:f285810f-2b70-4abb-afdd-db15790f17f1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "servicedPeriod": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + }, + "servicedPeriod": { + "start": "2012-05-15T09:41:37-07:00", + "end": "2012-05-15T10:41:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff", + "resource": { + "resourceType": "Encounter", + "id": "57add463-aac6-4cdb-9556-6e552d8c67ff", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "individual": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + } + ], + "period": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24", + "resource": { + "resourceType": "Condition", + "id": "f3f25715-3afa-4181-848b-c1b9c3ccaf24", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + }, + "onsetDateTime": "2012-07-02T09:41:37-07:00", + "abatementDateTime": "2012-07-09T09:41:37-07:00", + "recordedDate": "2012-07-02T09:41:37-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:a1a731b3-96ee-4935-b989-4f2db5f32a12", + "resource": { + "resourceType": "Procedure", + "id": "a1a731b3-96ee-4935-b989-4f2db5f32a12", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + }, + "performedPeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:02:37-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24", + "display": "Acute bronchitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d59314ed-ffb4-4bb9-8c31-37b46e7f6c39", + "resource": { + "resourceType": "MedicationRequest", + "id": "d59314ed-ffb4-4bb9-8c31-37b46e7f6c39", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1043400", + "display": "Acetaminophen 21.7 MG/ML / Dextromethorphan Hydrobromide 1 MG/ML / doxylamine succinate 0.417 MG/ML Oral Solution" + } + ], + "text": "Acetaminophen 21.7 MG/ML / Dextromethorphan Hydrobromide 1 MG/ML / doxylamine succinate 0.417 MG/ML Oral Solution" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + }, + "authoredOn": "2012-07-02T09:41:37-07:00", + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + }, + "reasonReference": [ + { + "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:05a6d60b-dadd-40c9-8505-17d16918446f", + "resource": { + "resourceType": "Claim", + "id": "05a6d60b-dadd-40c9-8505-17d16918446f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "created": "2012-07-02T10:17:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:d59314ed-ffb4-4bb9-8c31-37b46e7f6c39" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + } + ] + } + ], + "total": { + "value": 10.07, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9b2c09bd-bdda-4037-b00d-97c5faa0d735", + "resource": { + "resourceType": "CareTeam", + "id": "9b2c09bd-bdda-4037-b00d-97c5faa0d735", + "status": "inactive", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + }, + "period": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-12-31T08:41:37-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:433e5bff-790b-4f36-8ffd-f64cde69dc58", + "resource": { + "resourceType": "CarePlan", + "id": "433e5bff-790b-4f36-8ffd-f64cde69dc58", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + }, + "period": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-12-31T08:41:37-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:9b2c09bd-bdda-4037-b00d-97c5faa0d735" + } + ], + "addresses": [ + { + "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:0dba802b-36c4-4736-b27d-2e61ed03dd50", + "resource": { + "resourceType": "Claim", + "id": "0dba802b-36c4-4736-b27d-2e61ed03dd50", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "created": "2012-07-02T10:17:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a1a731b3-96ee-4935-b989-4f2db5f32a12" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8a2c0a9c-1e72-4839-96e1-e81ec44d5ed1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8a2c0a9c-1e72-4839-96e1-e81ec44d5ed1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "performer": [ + { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0dba802b-36c4-4736-b27d-2e61ed03dd50" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-07-02T10:17:37-07:00", + "end": "2013-07-02T10:17:37-07:00" + }, + "created": "2012-07-02T10:17:37-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0dba802b-36c4-4736-b27d-2e61ed03dd50" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "servicedPeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "servicedPeriod": { + "start": "2012-07-02T09:41:37-07:00", + "end": "2012-07-02T10:17:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "resource": { + "resourceType": "Organization", + "id": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "TRU MEDICAL WALK IN CENTER", + "telecom": [ + { + "system": "phone", + "value": "508-675-1522" + } + ], + "address": [ + { + "line": [ + "528 NEWTON STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "2721", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", + "resource": { + "resourceType": "Practitioner", + "id": "21a54030-cce5-3104-a773-9eb9bd60830b", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906799" + } + ], + "active": true, + "name": [ + { + "family": "Eichmann909", + "given": [ + "Wanetta360" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Wanetta360.Eichmann909@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "528 NEWTON STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "2721", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef", + "resource": { + "resourceType": "Encounter", + "id": "bd8588cc-ada9-4ac4-bcb0-49868d1c30ef", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-07-09T09:41:37-07:00", + "end": "2012-07-09T09:56:37-07:00" + }, + "individual": { + "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", + "display": "Dr. Wanetta360 Eichmann909" + } + } + ], + "period": { + "start": "2012-07-09T09:41:37-07:00", + "end": "2012-07-09T09:56:37-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "display": "TRU MEDICAL WALK IN CENTER" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b9ce49ce-a850-4fdc-8672-e0b222c42f94", + "resource": { + "resourceType": "Immunization", + "id": "b9ce49ce-a850-4fdc-8672-e0b222c42f94", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" + }, + "occurrenceDateTime": "2012-07-09T09:41:37-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b66781db-8de6-441f-ada7-a1304cf4cb0f", + "resource": { + "resourceType": "Claim", + "id": "b66781db-8de6-441f-ada7-a1304cf4cb0f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2012-07-09T09:41:37-07:00", + "end": "2012-07-09T09:56:37-07:00" + }, + "created": "2012-07-09T09:56:37-07:00", + "provider": { + "reference": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "display": "TRU MEDICAL WALK IN CENTER" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:b9ce49ce-a850-4fdc-8672-e0b222c42f94" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:de603239-8f97-4d7d-ba66-a35f1c4f99b0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "de603239-8f97-4d7d-ba66-a35f1c4f99b0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" + }, + "performer": [ + { + "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b66781db-8de6-441f-ada7-a1304cf4cb0f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-07-09T09:56:37-07:00", + "end": "2013-07-09T09:56:37-07:00" + }, + "created": "2012-07-09T09:56:37-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b66781db-8de6-441f-ada7-a1304cf4cb0f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + }, + "servicedPeriod": { + "start": "2012-07-09T09:41:37-07:00", + "end": "2012-07-09T09:56:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2012-07-09T09:41:37-07:00", + "end": "2012-07-09T09:56:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71", + "resource": { + "resourceType": "Encounter", + "id": "d44f687b-ec6d-400f-b784-1c61498adb71", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T09:11:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T09:11:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7c58963d-ef76-4731-ac96-008f443cdb97", + "resource": { + "resourceType": "Observation", + "id": "7c58963d-ef76-4731-ac96-008f443cdb97", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 131, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:48c40b57-7b57-4672-ac50-2abe158c0d71", + "resource": { + "resourceType": "Observation", + "id": "48c40b57-7b57-4672-ac50-2abe158c0d71", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db2345b6-c9d2-4774-921d-e9fbe6263d76", + "resource": { + "resourceType": "Observation", + "id": "db2345b6-c9d2-4774-921d-e9fbe6263d76", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 35.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8cd18bd4-4b06-4210-a5e1-379c4765e7b1", + "resource": { + "resourceType": "Observation", + "id": "8cd18bd4-4b06-4210-a5e1-379c4765e7b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 20.57, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a31a8e8-266e-45e8-a4f2-8a7d54686f6c", + "resource": { + "resourceType": "Observation", + "id": "1a31a8e8-266e-45e8-a4f2-8a7d54686f6c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 93.83, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8cda67d1-1480-42be-8aff-f0ea4268af9e", + "resource": { + "resourceType": "Observation", + "id": "8cda67d1-1480-42be-8aff-f0ea4268af9e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 131, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6f957375-798f-4922-a544-655d460d9a4d", + "resource": { + "resourceType": "Observation", + "id": "6f957375-798f-4922-a544-655d460d9a4d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 83, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:696fd53f-9007-4d8b-8b37-7092221d19c6", + "resource": { + "resourceType": "Observation", + "id": "696fd53f-9007-4d8b-8b37-7092221d19c6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:483b6520-0ec7-4b8d-bd4a-8622edd7cb4c", + "resource": { + "resourceType": "Observation", + "id": "483b6520-0ec7-4b8d-bd4a-8622edd7cb4c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "effectiveDateTime": "2012-12-31T08:41:37-08:00", + "issued": "2012-12-31T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36dc9d54-0462-4f5d-adfb-77a3cbeeebf9", + "resource": { + "resourceType": "Procedure", + "id": "36dc9d54-0462-4f5d-adfb-77a3cbeeebf9", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + }, + "performedPeriod": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T08:56:37-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3d1ef59c-db3f-417b-9c38-8d49f01fd2f2", + "resource": { + "resourceType": "Claim", + "id": "3d1ef59c-db3f-417b-9c38-8d49f01fd2f2", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T09:11:37-08:00" + }, + "created": "2012-12-31T09:11:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:36dc9d54-0462-4f5d-adfb-77a3cbeeebf9" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 799.48, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b020878c-ebb4-4192-a5f7-5fdb85be74fd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b020878c-ebb4-4192-a5f7-5fdb85be74fd", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3d1ef59c-db3f-417b-9c38-8d49f01fd2f2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2012-12-31T09:11:37-08:00", + "end": "2013-12-31T09:11:37-08:00" + }, + "created": "2012-12-31T09:11:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3d1ef59c-db3f-417b-9c38-8d49f01fd2f2" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2012-12-31T08:41:37-08:00", + "end": "2012-12-31T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 799.48, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 159.89600000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 639.5840000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 799.48, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 799.48, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 639.5840000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206", + "resource": { + "resourceType": "Encounter", + "id": "531fa623-ec6e-4e73-bff2-65ea7160f206", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-01-06T08:41:37-08:00", + "end": "2014-01-06T08:56:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2014-01-06T08:41:37-08:00", + "end": "2014-01-06T08:56:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:911f9129-1d1b-4c1f-9acc-15c27f191185", + "resource": { + "resourceType": "Observation", + "id": "911f9129-1d1b-4c1f-9acc-15c27f191185", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 135.9, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b2cdc4f-238f-49d8-aa58-211f5a9b78b3", + "resource": { + "resourceType": "Observation", + "id": "8b2cdc4f-238f-49d8-aa58-211f5a9b78b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bea7dae0-b453-4e64-8ab7-456037ff47c2", + "resource": { + "resourceType": "Observation", + "id": "bea7dae0-b453-4e64-8ab7-456037ff47c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 40.1, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9afd6319-2f22-4d1e-8554-b5d7d7d6a95a", + "resource": { + "resourceType": "Observation", + "id": "9afd6319-2f22-4d1e-8554-b5d7d7d6a95a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 21.72, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1c4ffe7-2f05-420f-ab2a-fbe5e974075f", + "resource": { + "resourceType": "Observation", + "id": "d1c4ffe7-2f05-420f-ab2a-fbe5e974075f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 94.162, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b08c7782-3d0d-4cba-aa50-5317d1e8470e", + "resource": { + "resourceType": "Observation", + "id": "b08c7782-3d0d-4cba-aa50-5317d1e8470e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 122, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2adb630-7077-451d-ae92-8e58274bf51b", + "resource": { + "resourceType": "Observation", + "id": "a2adb630-7077-451d-ae92-8e58274bf51b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 81, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca9ac15b-0d6d-4882-91b9-7b829398401e", + "resource": { + "resourceType": "Observation", + "id": "ca9ac15b-0d6d-4882-91b9-7b829398401e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e828e94-03a0-41aa-80fb-a265a296fe1d", + "resource": { + "resourceType": "Observation", + "id": "4e828e94-03a0-41aa-80fb-a265a296fe1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "effectiveDateTime": "2014-01-06T08:41:37-08:00", + "issued": "2014-01-06T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cad79faf-ed70-4afb-864b-c3b20c2a5c06", + "resource": { + "resourceType": "Immunization", + "id": "cad79faf-ed70-4afb-864b-c3b20c2a5c06", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + }, + "occurrenceDateTime": "2014-01-06T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e76109aa-f769-4340-ae67-6f9edbb24577", + "resource": { + "resourceType": "Claim", + "id": "e76109aa-f769-4340-ae67-6f9edbb24577", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2014-01-06T08:41:37-08:00", + "end": "2014-01-06T08:56:37-08:00" + }, + "created": "2014-01-06T08:56:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:cad79faf-ed70-4afb-864b-c3b20c2a5c06" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:063d2c05-1305-48d6-8a94-06a3355d164a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "063d2c05-1305-48d6-8a94-06a3355d164a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e76109aa-f769-4340-ae67-6f9edbb24577" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2014-01-06T08:56:37-08:00", + "end": "2015-01-06T08:56:37-08:00" + }, + "created": "2014-01-06T08:56:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e76109aa-f769-4340-ae67-6f9edbb24577" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2014-01-06T08:41:37-08:00", + "end": "2014-01-06T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2014-01-06T08:41:37-08:00", + "end": "2014-01-06T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6", + "resource": { + "resourceType": "Encounter", + "id": "55af649f-e26b-4710-b743-391f565d92f6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:762fe22e-568b-4838-b52b-2f3afec349b1", + "resource": { + "resourceType": "Observation", + "id": "762fe22e-568b-4838-b52b-2f3afec349b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 140.7, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e8cf17f-c9e2-4c54-ab73-b3f42a7b8c6b", + "resource": { + "resourceType": "Observation", + "id": "3e8cf17f-c9e2-4c54-ab73-b3f42a7b8c6b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6d79712-8bf9-4e82-83f8-c1b01ef824da", + "resource": { + "resourceType": "Observation", + "id": "c6d79712-8bf9-4e82-83f8-c1b01ef824da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 45.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0a275634-9c7c-4fa9-a5c0-4f6617b00f8c", + "resource": { + "resourceType": "Observation", + "id": "0a275634-9c7c-4fa9-a5c0-4f6617b00f8c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 22.92, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25a56f1c-b3e6-40a3-8206-6d31ed6ba574", + "resource": { + "resourceType": "Observation", + "id": "25a56f1c-b3e6-40a3-8206-6d31ed6ba574", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 94.498, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4da129ee-3bc5-436a-9948-6cab80d8ec1a", + "resource": { + "resourceType": "Observation", + "id": "4da129ee-3bc5-436a-9948-6cab80d8ec1a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 128, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:06957817-fdcc-421c-b791-44812931b932", + "resource": { + "resourceType": "Observation", + "id": "06957817-fdcc-421c-b791-44812931b932", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df847d91-3de2-40bb-9346-8654369d6ada", + "resource": { + "resourceType": "Observation", + "id": "df847d91-3de2-40bb-9346-8654369d6ada", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2a7e52a9-d189-43e1-9873-d8431e04a3d0", + "resource": { + "resourceType": "Observation", + "id": "2a7e52a9-d189-43e1-9873-d8431e04a3d0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 8.1514, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", + "resource": { + "resourceType": "Observation", + "id": "1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 4.5262, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6b92048-7c77-4773-b826-3639d7fcdf7c", + "resource": { + "resourceType": "Observation", + "id": "e6b92048-7c77-4773-b826-3639d7fcdf7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 13.159, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:948a0894-e573-4073-8c7b-97a26ba9edc2", + "resource": { + "resourceType": "Observation", + "id": "948a0894-e573-4073-8c7b-97a26ba9edc2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 35.212, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9b050885-8ea7-40d9-a9fa-02504f352846", + "resource": { + "resourceType": "Observation", + "id": "9b050885-8ea7-40d9-a9fa-02504f352846", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 92.463, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ea022d5-dd3f-436a-8d2d-2130e65d616e", + "resource": { + "resourceType": "Observation", + "id": "2ea022d5-dd3f-436a-8d2d-2130e65d616e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 27.982, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5aac06c7-9de2-4ff2-97cd-d1551ac83457", + "resource": { + "resourceType": "Observation", + "id": "5aac06c7-9de2-4ff2-97cd-d1551ac83457", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 34.283, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51622cb4-afab-4d28-993e-49286e3e9762", + "resource": { + "resourceType": "Observation", + "id": "51622cb4-afab-4d28-993e-49286e3e9762", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 39.384, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32e37758-5aa6-404b-bfa2-7e121d3599af", + "resource": { + "resourceType": "Observation", + "id": "32e37758-5aa6-404b-bfa2-7e121d3599af", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 358.98, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", + "resource": { + "resourceType": "Observation", + "id": "8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 295.93, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:426478df-b366-4f7c-ac2e-05bde99fe385", + "resource": { + "resourceType": "Observation", + "id": "426478df-b366-4f7c-ac2e-05bde99fe385", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueQuantity": { + "value": 9.4896, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e0cd77c-31e3-42a3-af98-36897c387b1f", + "resource": { + "resourceType": "Observation", + "id": "5e0cd77c-31e3-42a3-af98-36897c387b1f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7892a1fd-0d95-45e4-bea6-cc613b4a6e0d", + "resource": { + "resourceType": "Immunization", + "id": "7892a1fd-0d95-45e4-bea6-cc613b4a6e0d", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "115", + "display": "Tdap" + } + ], + "text": "Tdap" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "occurrenceDateTime": "2015-01-12T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8a6f3520-1ef7-4370-b54e-c6d87ca69fd4", + "resource": { + "resourceType": "Immunization", + "id": "8a6f3520-1ef7-4370-b54e-c6d87ca69fd4", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "occurrenceDateTime": "2015-01-12T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:eff3f274-7291-4850-9944-2247b963fd5c", + "resource": { + "resourceType": "Immunization", + "id": "eff3f274-7291-4850-9944-2247b963fd5c", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "occurrenceDateTime": "2015-01-12T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:98589c22-9e19-431f-a2fe-7156553c4d7e", + "resource": { + "resourceType": "Immunization", + "id": "98589c22-9e19-431f-a2fe-7156553c4d7e", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "occurrenceDateTime": "2015-01-12T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c5944ac4-b35d-44b7-b7a4-1a8eeec0c2d8", + "resource": { + "resourceType": "DiagnosticReport", + "id": "c5944ac4-b35d-44b7-b7a4-1a8eeec0c2d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + }, + "effectiveDateTime": "2015-01-12T08:41:37-08:00", + "issued": "2015-01-12T08:41:37.844-08:00", + "result": [ + { + "reference": "urn:uuid:2a7e52a9-d189-43e1-9873-d8431e04a3d0", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:e6b92048-7c77-4773-b826-3639d7fcdf7c", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:948a0894-e573-4073-8c7b-97a26ba9edc2", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:9b050885-8ea7-40d9-a9fa-02504f352846", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:2ea022d5-dd3f-436a-8d2d-2130e65d616e", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:5aac06c7-9de2-4ff2-97cd-d1551ac83457", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:51622cb4-afab-4d28-993e-49286e3e9762", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:32e37758-5aa6-404b-bfa2-7e121d3599af", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:426478df-b366-4f7c-ac2e-05bde99fe385", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:4fbfb167-03e9-46b4-890a-39d436328462", + "resource": { + "resourceType": "Claim", + "id": "4fbfb167-03e9-46b4-890a-39d436328462", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "created": "2015-01-12T08:56:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7892a1fd-0d95-45e4-bea6-cc613b4a6e0d" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8a6f3520-1ef7-4370-b54e-c6d87ca69fd4" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:eff3f274-7291-4850-9944-2247b963fd5c" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:98589c22-9e19-431f-a2fe-7156553c4d7e" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "115", + "display": "Tdap" + } + ], + "text": "Tdap" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 5, + "informationSequence": [ + 4 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f401b610-f7fb-4a70-b4ab-7f2a3ce950b8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f401b610-f7fb-4a70-b4ab-7f2a3ce950b8", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4fbfb167-03e9-46b4-890a-39d436328462" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2015-01-12T08:56:37-08:00", + "end": "2016-01-12T08:56:37-08:00" + }, + "created": "2015-01-12T08:56:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4fbfb167-03e9-46b4-890a-39d436328462" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "115", + "display": "Tdap" + } + ], + "text": "Tdap" + }, + "servicedPeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "servicedPeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "informationSequence": [ + 4 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "servicedPeriod": { + "start": "2015-01-12T08:41:37-08:00", + "end": "2015-01-12T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 449.66400000000004, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a", + "resource": { + "resourceType": "Encounter", + "id": "0b75bac0-fd68-44e7-8d14-9a6b8331ac2a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:da815f14-4818-4053-982a-50b3be7b6460", + "resource": { + "resourceType": "Observation", + "id": "da815f14-4818-4053-982a-50b3be7b6460", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 146, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8bae7e83-543a-4dea-8ad0-d53cab2009a4", + "resource": { + "resourceType": "Observation", + "id": "8bae7e83-543a-4dea-8ad0-d53cab2009a4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5fe9f9de-55ba-499a-8c16-e292600e153c", + "resource": { + "resourceType": "Observation", + "id": "5fe9f9de-55ba-499a-8c16-e292600e153c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 51.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49187e2c-56fc-44d8-98f9-41c94ca73632", + "resource": { + "resourceType": "Observation", + "id": "49187e2c-56fc-44d8-98f9-41c94ca73632", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 24.15, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d5f434bb-71ca-4b4f-aebe-dbb1ee4fda50", + "resource": { + "resourceType": "Observation", + "id": "d5f434bb-71ca-4b4f-aebe-dbb1ee4fda50", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 94.877, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ef2f58d6-526d-44c0-91ab-0234b58caad3", + "resource": { + "resourceType": "Observation", + "id": "ef2f58d6-526d-44c0-91ab-0234b58caad3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 120, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c7fb9fb1-6f9c-4842-8d4e-663e9b9991a0", + "resource": { + "resourceType": "Observation", + "id": "c7fb9fb1-6f9c-4842-8d4e-663e9b9991a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 69, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c1a567a-bc88-4bae-b4a5-0b4fb7ecbd15", + "resource": { + "resourceType": "Observation", + "id": "9c1a567a-bc88-4bae-b4a5-0b4fb7ecbd15", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:09edde80-b6ed-4d39-8bc9-47eeb6786ead", + "resource": { + "resourceType": "Observation", + "id": "09edde80-b6ed-4d39-8bc9-47eeb6786ead", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "effectiveDateTime": "2016-01-18T08:41:37-08:00", + "issued": "2016-01-18T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1854baa9-8629-49e5-a0f1-d03029099fa0", + "resource": { + "resourceType": "Immunization", + "id": "1854baa9-8629-49e5-a0f1-d03029099fa0", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "occurrenceDateTime": "2016-01-18T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:64fce6b8-8e84-4be2-8878-90e0b1716d26", + "resource": { + "resourceType": "Immunization", + "id": "64fce6b8-8e84-4be2-8878-90e0b1716d26", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + }, + "occurrenceDateTime": "2016-01-18T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7566ba8c-7330-42dd-94f6-2672e95e33e4", + "resource": { + "resourceType": "Claim", + "id": "7566ba8c-7330-42dd-94f6-2672e95e33e4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "created": "2016-01-18T08:56:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1854baa9-8629-49e5-a0f1-d03029099fa0" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:64fce6b8-8e84-4be2-8878-90e0b1716d26" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7d43aefc-7587-4b2e-9c73-572ec00d2c9b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7d43aefc-7587-4b2e-9c73-572ec00d2c9b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7566ba8c-7330-42dd-94f6-2672e95e33e4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2016-01-18T08:56:37-08:00", + "end": "2017-01-18T08:56:37-08:00" + }, + "created": "2016-01-18T08:56:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7566ba8c-7330-42dd-94f6-2672e95e33e4" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "servicedPeriod": { + "start": "2016-01-18T08:41:37-08:00", + "end": "2016-01-18T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 224.83200000000002, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99", + "resource": { + "resourceType": "Encounter", + "id": "b4c7f893-7782-42e5-9b39-6967e2e2fb99", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "individual": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + } + ], + "period": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e", + "resource": { + "resourceType": "Condition", + "id": "bc332dc7-b8c1-4a1f-9094-12b95e19ed3e", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + }, + "onsetDateTime": "2016-08-27T09:41:37-07:00", + "abatementDateTime": "2016-09-10T09:41:37-07:00", + "recordedDate": "2016-08-27T09:41:37-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:40ccbe9e-36a5-4c01-ab7a-e743c0a74766", + "resource": { + "resourceType": "MedicationRequest", + "id": "40ccbe9e-36a5-4c01-ab7a-e743c0a74766", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + }, + "authoredOn": "2016-08-27T09:41:37-07:00", + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:6db7fa8c-4dc3-4e06-8135-76d4a101871c", + "resource": { + "resourceType": "Claim", + "id": "6db7fa8c-4dc3-4e06-8135-76d4a101871c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "created": "2016-08-27T10:41:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:40ccbe9e-36a5-4c01-ab7a-e743c0a74766" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + } + ] + } + ], + "total": { + "value": 6.72, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f", + "resource": { + "resourceType": "CareTeam", + "id": "ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f", + "status": "inactive", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + }, + "period": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-09-10T09:41:37-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:a135cdd9-af11-4552-9f35-9a6dd6d952a9", + "resource": { + "resourceType": "CarePlan", + "id": "a135cdd9-af11-4552-9f35-9a6dd6d952a9", + "text": { + "status": "generated", + "div": "
    Care Plan for Physical therapy procedure.
    Activities:
    • Physical therapy procedure
    • Physical therapy procedure

    Care plan is meant to treat Sprain of ankle.
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "91251008", + "display": "Physical therapy procedure" + } + ], + "text": "Physical therapy procedure" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + }, + "period": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-09-10T09:41:37-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f" + } + ], + "addresses": [ + { + "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" + }, + { + "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229586001", + "display": "Rest, ice, compression and elevation treatment programme" + } + ], + "text": "Rest, ice, compression and elevation treatment programme" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229070002", + "display": "Stretching exercises" + } + ], + "text": "Stretching exercises" + }, + "status": "completed", + "location": { + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:e102d5db-4bf7-4fbe-8dc7-315a366ac4cb", + "resource": { + "resourceType": "Claim", + "id": "e102d5db-4bf7-4fbe-8dc7-315a366ac4cb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "created": "2016-08-27T10:41:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f28cfb37-5d1f-4d44-9176-65343a68862f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f28cfb37-5d1f-4d44-9176-65343a68862f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "performer": [ + { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e102d5db-4bf7-4fbe-8dc7-315a366ac4cb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2016-08-27T10:41:37-07:00", + "end": "2017-08-27T10:41:37-07:00" + }, + "created": "2016-08-27T10:41:37-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e102d5db-4bf7-4fbe-8dc7-315a366ac4cb" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "servicedPeriod": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + }, + "servicedPeriod": { + "start": "2016-08-27T09:41:37-07:00", + "end": "2016-08-27T10:41:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758", + "resource": { + "resourceType": "Encounter", + "id": "a08c5912-933e-4da3-8100-31264995b758", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:60bf9346-3ae2-45a0-8149-3cdf03bbe7d7", + "resource": { + "resourceType": "Observation", + "id": "60bf9346-3ae2-45a0-8149-3cdf03bbe7d7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 153, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d410a58a-3c95-4007-ab5a-6da844a168ef", + "resource": { + "resourceType": "Observation", + "id": "d410a58a-3c95-4007-ab5a-6da844a168ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d890600-5da7-4eaa-8bca-e5da92d11a4e", + "resource": { + "resourceType": "Observation", + "id": "1d890600-5da7-4eaa-8bca-e5da92d11a4e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 64.9, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69acd0f8-b26c-4bf3-bc27-c253cf448694", + "resource": { + "resourceType": "Observation", + "id": "69acd0f8-b26c-4bf3-bc27-c253cf448694", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 27.73, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:039a00ad-eb92-490a-83da-4b71ed0a2698", + "resource": { + "resourceType": "Observation", + "id": "039a00ad-eb92-490a-83da-4b71ed0a2698", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 97.488, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f2f9a723-0f49-4672-b479-ced8a850cf19", + "resource": { + "resourceType": "Observation", + "id": "f2f9a723-0f49-4672-b479-ced8a850cf19", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 115, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69a22dae-c5c7-4da0-8c61-bb3944504c31", + "resource": { + "resourceType": "Observation", + "id": "69a22dae-c5c7-4da0-8c61-bb3944504c31", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 64, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6068bc02-26f0-432f-9ff8-c4bcdbc414f2", + "resource": { + "resourceType": "Observation", + "id": "6068bc02-26f0-432f-9ff8-c4bcdbc414f2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9fa3128-43e2-4ea6-9900-0c73c826d4bc", + "resource": { + "resourceType": "Observation", + "id": "c9fa3128-43e2-4ea6-9900-0c73c826d4bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "effectiveDateTime": "2017-01-23T08:41:37-08:00", + "issued": "2017-01-23T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e965c8e-bcc9-4c3a-be01-ecafbf4f6094", + "resource": { + "resourceType": "Procedure", + "id": "0e965c8e-bcc9-4c3a-be01-ecafbf4f6094", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "performedPeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T08:56:37-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:95c09b71-8c04-4778-922a-c5d207e7c989", + "resource": { + "resourceType": "Immunization", + "id": "95c09b71-8c04-4778-922a-c5d207e7c989", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "occurrenceDateTime": "2017-01-23T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e3b92940-d554-47b0-ac36-62bcec08f3dc", + "resource": { + "resourceType": "Immunization", + "id": "e3b92940-d554-47b0-ac36-62bcec08f3dc", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + }, + "occurrenceDateTime": "2017-01-23T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a488e6d7-7b1a-4383-a5b2-fc61327f9fd0", + "resource": { + "resourceType": "Claim", + "id": "a488e6d7-7b1a-4383-a5b2-fc61327f9fd0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "created": "2017-01-23T09:11:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:95c09b71-8c04-4778-922a-c5d207e7c989" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e3b92940-d554-47b0-ac36-62bcec08f3dc" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:0e965c8e-bcc9-4c3a-be01-ecafbf4f6094" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 577.47, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:410f2644-845c-4d89-9dbd-b712c4bf8f3b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "410f2644-845c-4d89-9dbd-b712c4bf8f3b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a488e6d7-7b1a-4383-a5b2-fc61327f9fd0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2017-01-23T09:11:37-08:00", + "end": "2018-01-23T09:11:37-08:00" + }, + "created": "2017-01-23T09:11:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a488e6d7-7b1a-4383-a5b2-fc61327f9fd0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "servicedPeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T08:41:37-08:00", + "end": "2017-01-23T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 577.47, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 115.49400000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 461.97600000000006, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 577.47, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 577.47, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 686.8080000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695", + "resource": { + "resourceType": "Encounter", + "id": "2f7612c4-f9c3-4343-90c5-ae3e8e9c9695", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a", + "resource": { + "resourceType": "Condition", + "id": "9b1f80d9-3e7a-49ba-9371-102c0af0803a", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "onsetDateTime": "2018-01-29T08:41:37-08:00", + "recordedDate": "2018-01-29T08:41:37-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:054fc06a-e893-4090-8688-1b400253524f", + "resource": { + "resourceType": "Observation", + "id": "054fc06a-e893-4090-8688-1b400253524f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 161.2, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d78804de-9030-4fe2-8173-9c729ab82186", + "resource": { + "resourceType": "Observation", + "id": "d78804de-9030-4fe2-8173-9c729ab82186", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d57a613-7f09-4a56-9d45-3476a8a6edc1", + "resource": { + "resourceType": "Observation", + "id": "5d57a613-7f09-4a56-9d45-3476a8a6edc1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 79.2, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:093bb0b2-cfc0-4e0b-9b4c-5a1a4eddddc3", + "resource": { + "resourceType": "Observation", + "id": "093bb0b2-cfc0-4e0b-9b4c-5a1a4eddddc3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 30.49, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:398ea67e-6187-4760-8e99-c757d5d4148a", + "resource": { + "resourceType": "Observation", + "id": "398ea67e-6187-4760-8e99-c757d5d4148a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 98.364, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4245b809-6791-4fc6-aea1-55a0989fd59e", + "resource": { + "resourceType": "Observation", + "id": "4245b809-6791-4fc6-aea1-55a0989fd59e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 115, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7104eb83-3044-4b45-ad7b-67cd4891f1ef", + "resource": { + "resourceType": "Observation", + "id": "7104eb83-3044-4b45-ad7b-67cd4891f1ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b76f04f1-aa2f-4d0b-aea0-8b08012dd09f", + "resource": { + "resourceType": "Observation", + "id": "b76f04f1-aa2f-4d0b-aea0-8b08012dd09f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:58510814-1edf-44a6-8fbf-43171f1a0f30", + "resource": { + "resourceType": "Observation", + "id": "58510814-1edf-44a6-8fbf-43171f1a0f30", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "effectiveDateTime": "2018-01-29T08:41:37-08:00", + "issued": "2018-01-29T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e755db2e-b8a7-4b64-8fa3-ebf09625571c", + "resource": { + "resourceType": "Immunization", + "id": "e755db2e-b8a7-4b64-8fa3-ebf09625571c", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + }, + "occurrenceDateTime": "2018-01-29T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:feed7acd-8ff0-42cc-b0da-725a1b513d90", + "resource": { + "resourceType": "Claim", + "id": "feed7acd-8ff0-42cc-b0da-725a1b513d90", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "created": "2018-01-29T08:56:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e755db2e-b8a7-4b64-8fa3-ebf09625571c" + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e55a3e5c-8524-4a1a-93bc-5c13b3b71b74", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e55a3e5c-8524-4a1a-93bc-5c13b3b71b74", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "feed7acd-8ff0-42cc-b0da-725a1b513d90" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2018-01-29T08:56:37-08:00", + "end": "2019-01-29T08:56:37-08:00" + }, + "created": "2018-01-29T08:56:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:feed7acd-8ff0-42cc-b0da-725a1b513d90" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "servicedPeriod": { + "start": "2018-01-29T08:41:37-08:00", + "end": "2018-01-29T08:56:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1", + "resource": { + "resourceType": "Encounter", + "id": "e1157598-1cdd-4242-812d-a3ab228a69e1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "individual": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + } + } + ], + "period": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f", + "resource": { + "resourceType": "Condition", + "id": "b913dddb-6e54-452e-9559-7fdfd7356a1f", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + }, + "onsetDateTime": "2018-10-05T09:41:37-07:00", + "abatementDateTime": "2018-10-14T09:41:37-07:00", + "recordedDate": "2018-10-05T09:41:37-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b830c0e1-09fe-4ea9-b55c-fef6cebaa94b", + "resource": { + "resourceType": "Observation", + "id": "b830c0e1-09fe-4ea9-b55c-fef6cebaa94b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + }, + { + "system": "http://loinc.org", + "code": "8331-1", + "display": "Oral temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + }, + "effectiveDateTime": "2018-10-05T09:41:37-07:00", + "issued": "2018-10-05T09:41:37.844-07:00", + "valueQuantity": { + "value": 39.25, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70", + "resource": { + "resourceType": "MedicationRequest", + "id": "7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "834061", + "display": "Penicillin V Potassium 250 MG Oral Tablet" + } + ], + "text": "Penicillin V Potassium 250 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + }, + "authoredOn": "2018-10-05T09:41:37-07:00", + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "display": "Dr. Gillian484 Brakus656" + }, + "reasonReference": [ + { + "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1aef6704-cada-4b33-b7e3-b5f6c16389a1", + "resource": { + "resourceType": "Claim", + "id": "1aef6704-cada-4b33-b7e3-b5f6c16389a1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "created": "2018-10-05T09:56:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + } + ] + } + ], + "total": { + "value": 18.31, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7cabe084-1ba9-43cc-bddb-3e34b858abf7", + "resource": { + "resourceType": "Claim", + "id": "7cabe084-1ba9-43cc-bddb-3e34b858abf7", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "created": "2018-10-05T09:56:37-07:00", + "provider": { + "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "display": "SOUTHCOAST HOSPITAL GROUP, INC" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:71dc2f89-9a20-4f31-85e4-ef582fc534aa", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "71dc2f89-9a20-4f31-85e4-ef582fc534aa", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "performer": [ + { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7cabe084-1ba9-43cc-bddb-3e34b858abf7" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2018-10-05T09:56:37-07:00", + "end": "2019-10-05T09:56:37-07:00" + }, + "created": "2018-10-05T09:56:37-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7cabe084-1ba9-43cc-bddb-3e34b858abf7" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + }, + "servicedPeriod": { + "start": "2018-10-05T09:41:37-07:00", + "end": "2018-10-05T09:56:37-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1", + "resource": { + "resourceType": "Encounter", + "id": "d546dd8a-a78c-4aa8-a99e-74a15f7b88f1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:37e44604-1853-4981-96c4-e191c83cf6d5", + "resource": { + "resourceType": "Observation", + "id": "37e44604-1853-4981-96c4-e191c83cf6d5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 167.1, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:06af490c-2c29-4faf-b826-2167c92aaddf", + "resource": { + "resourceType": "Observation", + "id": "06af490c-2c29-4faf-b826-2167c92aaddf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9a50148c-562b-43e4-8d61-9634b548be88", + "resource": { + "resourceType": "Observation", + "id": "9a50148c-562b-43e4-8d61-9634b548be88", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 85.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b2a5179-75a4-4778-bc44-661f1b62081f", + "resource": { + "resourceType": "Observation", + "id": "3b2a5179-75a4-4778-bc44-661f1b62081f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 30.56, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40589108-8442-4dfe-8bb8-4108695c07c3", + "resource": { + "resourceType": "Observation", + "id": "40589108-8442-4dfe-8bb8-4108695c07c3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 98.13, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e1e077a3-8241-4ad7-a0b4-a4a6ae0c5fe6", + "resource": { + "resourceType": "Observation", + "id": "e1e077a3-8241-4ad7-a0b4-a4a6ae0c5fe6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 124, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:299a9fbc-f0bd-4dfe-a307-d740cfbbbb36", + "resource": { + "resourceType": "Observation", + "id": "299a9fbc-f0bd-4dfe-a307-d740cfbbbb36", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 77, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:faace313-4b23-4b02-ad30-3d319503709c", + "resource": { + "resourceType": "Observation", + "id": "faace313-4b23-4b02-ad30-3d319503709c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:997e98bd-14cc-4593-98b4-8efbe0a1995a", + "resource": { + "resourceType": "Observation", + "id": "997e98bd-14cc-4593-98b4-8efbe0a1995a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "effectiveDateTime": "2019-02-04T08:41:37-08:00", + "issued": "2019-02-04T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8839553d-b467-4604-a905-0a54bff6127c", + "resource": { + "resourceType": "Procedure", + "id": "8839553d-b467-4604-a905-0a54bff6127c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "performedPeriod": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T08:56:37-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c2d91b68-558e-407f-958e-14fdb79c13e9", + "resource": { + "resourceType": "Immunization", + "id": "c2d91b68-558e-407f-958e-14fdb79c13e9", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + }, + "occurrenceDateTime": "2019-02-04T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4d860d59-230f-408f-a8bd-d5603283fd7e", + "resource": { + "resourceType": "Claim", + "id": "4d860d59-230f-408f-a8bd-d5603283fd7e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "created": "2019-02-04T09:11:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c2d91b68-558e-407f-958e-14fdb79c13e9" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:8839553d-b467-4604-a905-0a54bff6127c" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 747.00, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:00dbcc1e-4d4a-4372-b048-1fc72717b155", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "00dbcc1e-4d4a-4372-b048-1fc72717b155", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4d860d59-230f-408f-a8bd-d5603283fd7e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2019-02-04T09:11:37-08:00", + "end": "2020-02-04T09:11:37-08:00" + }, + "created": "2019-02-04T09:11:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4d860d59-230f-408f-a8bd-d5603283fd7e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2019-02-04T08:41:37-08:00", + "end": "2019-02-04T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 747.00, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 149.4, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 597.6, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 747.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 747.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 710.0160000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48", + "resource": { + "resourceType": "Encounter", + "id": "1187f024-a959-4ab6-87b5-8ffe5af07b48", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "individual": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "display": "Dr. Bertie593 Mante251" + } + } + ], + "period": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d7b4d6c6-b9db-497e-9406-fc249ad26929", + "resource": { + "resourceType": "Observation", + "id": "d7b4d6c6-b9db-497e-9406-fc249ad26929", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 170.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f9400dde-b562-4673-9e42-1d3c3fb7174f", + "resource": { + "resourceType": "Observation", + "id": "f9400dde-b562-4673-9e42-1d3c3fb7174f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:904aac17-b1a0-4e08-8856-cd49813d62a9", + "resource": { + "resourceType": "Observation", + "id": "904aac17-b1a0-4e08-8856-cd49813d62a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 79.4, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ea6e3c7-072c-4c64-af04-a182ab3d4de1", + "resource": { + "resourceType": "Observation", + "id": "9ea6e3c7-072c-4c64-af04-a182ab3d4de1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 27.31, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5b41f610-059c-4cbe-81c6-2d273903a7a7", + "resource": { + "resourceType": "Observation", + "id": "5b41f610-059c-4cbe-81c6-2d273903a7a7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 94.495, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db83d79b-829a-4caf-b4c4-5f75f43e6737", + "resource": { + "resourceType": "Observation", + "id": "db83d79b-829a-4caf-b4c4-5f75f43e6737", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 88, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 102, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:044db950-91d6-410b-9420-7a76fc0be37a", + "resource": { + "resourceType": "Observation", + "id": "044db950-91d6-410b-9420-7a76fc0be37a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 75, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a32d8cb1-80c8-43ba-abfd-c8d0b4d6a915", + "resource": { + "resourceType": "Observation", + "id": "a32d8cb1-80c8-43ba-abfd-c8d0b4d6a915", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", + "resource": { + "resourceType": "Observation", + "id": "3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 8.7281, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28bb65d6-cb96-4d98-b4f2-335ace569991", + "resource": { + "resourceType": "Observation", + "id": "28bb65d6-cb96-4d98-b4f2-335ace569991", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 4.8239, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", + "resource": { + "resourceType": "Observation", + "id": "b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 12.454, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a45d891-738f-4a86-85ba-e97c718ed4b2", + "resource": { + "resourceType": "Observation", + "id": "1a45d891-738f-4a86-85ba-e97c718ed4b2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 36.323, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5cf721e8-9e25-470a-a19a-0389b4b12213", + "resource": { + "resourceType": "Observation", + "id": "5cf721e8-9e25-470a-a19a-0389b4b12213", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 84.472, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:514bd901-b8ac-4871-8b99-d86db24273a9", + "resource": { + "resourceType": "Observation", + "id": "514bd901-b8ac-4871-8b99-d86db24273a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 31.404, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:07f26a58-f3b6-46d1-8d4c-88927bf98ebb", + "resource": { + "resourceType": "Observation", + "id": "07f26a58-f3b6-46d1-8d4c-88927bf98ebb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 34.657, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4174786-f2ce-45cb-bd39-6f70e6d60cb7", + "resource": { + "resourceType": "Observation", + "id": "d4174786-f2ce-45cb-bd39-6f70e6d60cb7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 39.701, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67074975-d413-4fd7-965c-5db34b174ed0", + "resource": { + "resourceType": "Observation", + "id": "67074975-d413-4fd7-965c-5db34b174ed0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 317.45, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71b1e784-b609-4186-9352-32874dbf1fcb", + "resource": { + "resourceType": "Observation", + "id": "71b1e784-b609-4186-9352-32874dbf1fcb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 364.02, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a03734f2-8ff9-4bf3-bb56-e049c409269e", + "resource": { + "resourceType": "Observation", + "id": "a03734f2-8ff9-4bf3-bb56-e049c409269e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueQuantity": { + "value": 11.938, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4bbe85dc-c5f6-4b8b-9f56-ace4e35d7b2a", + "resource": { + "resourceType": "Observation", + "id": "4bbe85dc-c5f6-4b8b-9f56-ace4e35d7b2a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2342c6a-f2b3-49cb-a259-5214d0246872", + "resource": { + "resourceType": "Procedure", + "id": "e2342c6a-f2b3-49cb-a259-5214d0246872", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "performedPeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T08:56:37-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f", + "resource": { + "resourceType": "Immunization", + "id": "0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "occurrenceDateTime": "2020-02-10T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c862abe2-d093-488e-a216-d869790e3d34", + "resource": { + "resourceType": "Immunization", + "id": "c862abe2-d093-488e-a216-d869790e3d34", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "occurrenceDateTime": "2020-02-10T08:41:37-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0404d41c-01f9-4b4e-866d-30d8e1c87768", + "resource": { + "resourceType": "DiagnosticReport", + "id": "0404d41c-01f9-4b4e-866d-30d8e1c87768", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "encounter": { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + }, + "effectiveDateTime": "2020-02-10T08:41:37-08:00", + "issued": "2020-02-10T08:41:37.844-08:00", + "result": [ + { + "reference": "urn:uuid:3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:28bb65d6-cb96-4d98-b4f2-335ace569991", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:1a45d891-738f-4a86-85ba-e97c718ed4b2", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:5cf721e8-9e25-470a-a19a-0389b4b12213", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:514bd901-b8ac-4871-8b99-d86db24273a9", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:07f26a58-f3b6-46d1-8d4c-88927bf98ebb", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:d4174786-f2ce-45cb-bd39-6f70e6d60cb7", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:67074975-d413-4fd7-965c-5db34b174ed0", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:71b1e784-b609-4186-9352-32874dbf1fcb", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:a03734f2-8ff9-4bf3-bb56-e049c409269e", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:faa1c861-d568-4d37-a9be-e14f02aabbe5", + "resource": { + "resourceType": "Claim", + "id": "faa1c861-d568-4d37-a9be-e14f02aabbe5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", + "display": "Norberto865 Cole117" + }, + "billablePeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "created": "2020-02-10T09:11:37-08:00", + "provider": { + "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "display": "PCP235727" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c862abe2-d093-488e-a216-d869790e3d34" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e2342c6a-f2b3-49cb-a259-5214d0246872" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 730.56, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c6949f8d-cb69-4a44-9186-8b336f06dfd4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c6949f8d-cb69-4a44-9186-8b336f06dfd4", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "requester": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "performer": [ + { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "faa1c861-d568-4d37-a9be-e14f02aabbe5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" + }, + "billablePeriod": { + "start": "2020-02-10T09:11:37-08:00", + "end": "2021-02-10T09:11:37-08:00" + }, + "created": "2020-02-10T09:11:37-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:faa1c861-d568-4d37-a9be-e14f02aabbe5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + }, + "servicedPeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "servicedPeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2020-02-10T08:41:37-08:00", + "end": "2020-02-10T09:11:37-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 730.56, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 146.112, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 584.448, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 730.56, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 730.56, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 809.28, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json new file mode 100644 index 000000000000..833a083e3d58 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json @@ -0,0 +1,80348 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "resource": { + "resourceType": "Patient", + "id": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -8362707934077662551 Population seed: 1586368870505
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Clelia416 Schmitt836" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", + "valueAddress": { + "city": "Fall River", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 2.806179064509155 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 70.19382093549085 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "SS", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-23-6926" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99931018" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "PPN", + "display": "Passport Number" + } + ], + "text": "Passport Number" + }, + "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", + "value": "X38526821X" + } + ], + "name": [ + { + "use": "official", + "family": "DuBuque211", + "given": [ + "Robbyn526" + ], + "prefix": [ + "Mrs." + ] + }, + { + "use": "maiden", + "family": "Herzog843", + "given": [ + "Robbyn526" + ], + "prefix": [ + "Mrs." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-953-9961", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1946-12-09", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 41.7460901626166 + }, + { + "url": "longitude", + "valueDecimal": -70.7291724846944 + } + ] + } + ], + "line": [ + "149 Conroy Corner Unit 5" + ], + "city": "Wareham", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", + "code": "M", + "display": "M" + } + ], + "text": "M" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "resource": { + "resourceType": "Organization", + "id": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP128586", + "address": [ + { + "line": [ + "2360 CRANBERRY HWY" + ], + "city": "W WAREHAM", + "state": "MA", + "postalCode": "02576-1208", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "resource": { + "resourceType": "Practitioner", + "id": "cf99bf43-51e5-3bab-8623-211cd48f66dc", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999951739" + } + ], + "active": true, + "name": [ + { + "family": "DuBuque211", + "given": [ + "Carlyn477" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Carlyn477.DuBuque211@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "2360 CRANBERRY HWY" + ], + "city": "W WAREHAM", + "state": "MA", + "postalCode": "02576-1208", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3", + "resource": { + "resourceType": "Encounter", + "id": "24d07440-757c-4b8a-9df9-c8bf9e70a6d3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1965-02-01T04:15:51-08:00", + "end": "1965-02-01T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1965-02-01T04:15:51-08:00", + "end": "1965-02-01T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad", + "resource": { + "resourceType": "Condition", + "id": "023eafd5-a008-4f7e-b974-8e2a25e9e3ad", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "59621000", + "display": "Hypertension" + } + ], + "text": "Hypertension" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" + }, + "onsetDateTime": "1965-02-01T04:15:51-08:00", + "recordedDate": "1965-02-01T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:a7e673c4-df3d-4141-8541-9d494b613bb9", + "resource": { + "resourceType": "CareTeam", + "id": "a7e673c4-df3d-4141-8541-9d494b613bb9", + "status": "active", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" + }, + "period": { + "start": "1965-02-01T04:15:51-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "59621000", + "display": "Hypertension" + } + ], + "text": "Hypertension" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:b10b776c-fe84-4b21-9367-8e79d5b2d945", + "resource": { + "resourceType": "CarePlan", + "id": "b10b776c-fe84-4b21-9367-8e79d5b2d945", + "text": { + "status": "generated", + "div": "
    Care Plan for Lifestyle education regarding hypertension.
    Activities:
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension

    Care plan is meant to treat Hypertension.
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "443402002", + "display": "Lifestyle education regarding hypertension" + } + ], + "text": "Lifestyle education regarding hypertension" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" + }, + "period": { + "start": "1965-02-01T04:15:51-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:a7e673c4-df3d-4141-8541-9d494b613bb9" + } + ], + "addresses": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "386463000", + "display": "Prescribed activity/exercise education" + } + ], + "text": "Prescribed activity/exercise education" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "413473000", + "display": "Counseling about alcohol consumption" + } + ], + "text": "Counseling about alcohol consumption" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "1151000175103", + "display": "Dietary approaches to stop hypertension diet" + } + ], + "text": "Dietary approaches to stop hypertension diet" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225323000", + "display": "Smoking cessation education" + } + ], + "text": "Smoking cessation education" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:ff4fe9f1-2aef-499b-a72b-809b0d83f747", + "resource": { + "resourceType": "Claim", + "id": "ff4fe9f1-2aef-499b-a72b-809b0d83f747", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1965-02-01T04:15:51-08:00", + "end": "1965-02-01T04:45:51-08:00" + }, + "created": "1965-02-01T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "59621000", + "display": "Hypertension" + } + ], + "text": "Hypertension" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ecebe2d4-1fc3-4819-a5e0-99d6592f74ce", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ecebe2d4-1fc3-4819-a5e0-99d6592f74ce", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Anthem" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Anthem" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ff4fe9f1-2aef-499b-a72b-809b0d83f747" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1965-02-01T04:45:51-08:00", + "end": "1966-02-01T04:45:51-08:00" + }, + "created": "1965-02-01T04:45:51-08:00", + "insurer": { + "display": "Anthem" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ff4fe9f1-2aef-499b-a72b-809b0d83f747" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1965-02-01T04:15:51-08:00", + "end": "1965-02-01T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "59621000", + "display": "Hypertension" + } + ], + "text": "Hypertension" + }, + "servicedPeriod": { + "start": "1965-02-01T04:15:51-08:00", + "end": "1965-02-01T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "resource": { + "resourceType": "Organization", + "id": "ef6ab57c-ed94-3dbe-9861-812d515918b3", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ef6ab57c-ed94-3dbe-9861-812d515918b3" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAPE COD HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5087711800" + } + ], + "address": [ + { + "line": [ + "88 LEWIS BAY ROAD" + ], + "city": "HYANNIS", + "state": "MA", + "postalCode": "02601", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "resource": { + "resourceType": "Practitioner", + "id": "fb3b949e-e3cb-3069-b280-521ac0512d2e", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999949" + } + ], + "active": true, + "name": [ + { + "family": "Kautzer186", + "given": [ + "Heath320" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Heath320.Kautzer186@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "88 LEWIS BAY ROAD" + ], + "city": "HYANNIS", + "state": "MA", + "postalCode": "02601", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9", + "resource": { + "resourceType": "Encounter", + "id": "4e1a4ce2-7a3c-404f-b744-14309ed21fb9", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1965-03-03T04:15:51-08:00", + "end": "1965-03-03T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1965-03-03T04:15:51-08:00", + "end": "1965-03-03T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0f0e3847-3669-4b37-b635-4ffb0c090bab", + "resource": { + "resourceType": "MedicationRequest", + "id": "0f0e3847-3669-4b37-b635-4ffb0c090bab", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" + }, + "authoredOn": "1965-03-03T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:acf2a21e-53ba-4eea-9976-d6f51576cf10", + "resource": { + "resourceType": "Claim", + "id": "acf2a21e-53ba-4eea-9976-d6f51576cf10", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1965-03-03T04:15:51-08:00", + "end": "1965-03-03T04:30:51-08:00" + }, + "created": "1965-03-03T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0f0e3847-3669-4b37-b635-4ffb0c090bab" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + }, + "encounter": [ + { + "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0a11b721-5c8a-4b1b-a41c-6619a256c242", + "resource": { + "resourceType": "Claim", + "id": "0a11b721-5c8a-4b1b-a41c-6619a256c242", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1965-03-03T04:15:51-08:00", + "end": "1965-03-03T04:30:51-08:00" + }, + "created": "1965-03-03T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + }, + "encounter": [ + { + "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:607389a1-f751-4937-ac5c-bd61e8792def", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "607389a1-f751-4937-ac5c-bd61e8792def", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0a11b721-5c8a-4b1b-a41c-6619a256c242" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1965-03-03T04:30:51-08:00", + "end": "1966-03-03T04:30:51-08:00" + }, + "created": "1965-03-03T04:30:51-08:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0a11b721-5c8a-4b1b-a41c-6619a256c242" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + }, + "servicedPeriod": { + "start": "1965-03-03T04:15:51-08:00", + "end": "1965-03-03T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c", + "resource": { + "resourceType": "Encounter", + "id": "754fecdd-c8c2-4387-9e98-e3680ce6960c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1966-02-07T04:15:51-08:00", + "end": "1966-02-07T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1966-02-07T04:15:51-08:00", + "end": "1966-02-07T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:861d1251-de70-41ec-b9ee-d11ccc175f8f", + "resource": { + "resourceType": "MedicationRequest", + "id": "861d1251-de70-41ec-b9ee-d11ccc175f8f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" + }, + "authoredOn": "1966-02-07T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:7d3303e6-8f8d-4b60-9440-cebd425c778e", + "resource": { + "resourceType": "Claim", + "id": "7d3303e6-8f8d-4b60-9440-cebd425c778e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1966-02-07T04:15:51-08:00", + "end": "1966-02-07T04:45:51-08:00" + }, + "created": "1966-02-07T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:861d1251-de70-41ec-b9ee-d11ccc175f8f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:03abe32d-1671-47a7-ab71-11ca6b50668c", + "resource": { + "resourceType": "Claim", + "id": "03abe32d-1671-47a7-ab71-11ca6b50668c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1966-02-07T04:15:51-08:00", + "end": "1966-02-07T04:45:51-08:00" + }, + "created": "1966-02-07T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ca9a77aa-90f8-4c7c-bbd1-5776630b6f9f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ca9a77aa-90f8-4c7c-bbd1-5776630b6f9f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Anthem" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Anthem" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "03abe32d-1671-47a7-ab71-11ca6b50668c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1966-02-07T04:45:51-08:00", + "end": "1967-02-07T04:45:51-08:00" + }, + "created": "1966-02-07T04:45:51-08:00", + "insurer": { + "display": "Anthem" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:03abe32d-1671-47a7-ab71-11ca6b50668c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1966-02-07T04:15:51-08:00", + "end": "1966-02-07T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0", + "resource": { + "resourceType": "Encounter", + "id": "692ff055-48d8-40d0-a4d7-7b3df71a0cb0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1967-02-13T04:15:51-08:00", + "end": "1967-02-13T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1967-02-13T04:15:51-08:00", + "end": "1967-02-13T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7be2a83b-078e-475f-90f0-943b67406ea8", + "resource": { + "resourceType": "MedicationRequest", + "id": "7be2a83b-078e-475f-90f0-943b67406ea8", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" + }, + "authoredOn": "1967-02-13T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:358d194c-0f8f-47a1-925f-960a99387816", + "resource": { + "resourceType": "Claim", + "id": "358d194c-0f8f-47a1-925f-960a99387816", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1967-02-13T04:15:51-08:00", + "end": "1967-02-13T04:30:51-08:00" + }, + "created": "1967-02-13T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:7be2a83b-078e-475f-90f0-943b67406ea8" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2cc2ffea-829c-472f-b75a-e8365aedc49f", + "resource": { + "resourceType": "Claim", + "id": "2cc2ffea-829c-472f-b75a-e8365aedc49f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1967-02-13T04:15:51-08:00", + "end": "1967-02-13T04:30:51-08:00" + }, + "created": "1967-02-13T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e87339f3-9111-4406-b0a8-ed7f05ac82d2", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e87339f3-9111-4406-b0a8-ed7f05ac82d2", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Anthem" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Anthem" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2cc2ffea-829c-472f-b75a-e8365aedc49f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1967-02-13T04:30:51-08:00", + "end": "1968-02-13T04:30:51-08:00" + }, + "created": "1967-02-13T04:30:51-08:00", + "insurer": { + "display": "Anthem" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2cc2ffea-829c-472f-b75a-e8365aedc49f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1967-02-13T04:15:51-08:00", + "end": "1967-02-13T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4", + "resource": { + "resourceType": "Encounter", + "id": "927ebc18-b85d-4216-8997-85d27747d0f4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1968-02-19T04:15:51-08:00", + "end": "1968-02-19T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1968-02-19T04:15:51-08:00", + "end": "1968-02-19T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0f3ac464-fc33-4fce-bb09-6a8f5b0cc421", + "resource": { + "resourceType": "MedicationRequest", + "id": "0f3ac464-fc33-4fce-bb09-6a8f5b0cc421", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" + }, + "authoredOn": "1968-02-19T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:2ca0b36a-5aa9-4806-9852-457615c05e40", + "resource": { + "resourceType": "Claim", + "id": "2ca0b36a-5aa9-4806-9852-457615c05e40", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1968-02-19T04:15:51-08:00", + "end": "1968-02-19T04:30:51-08:00" + }, + "created": "1968-02-19T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0f3ac464-fc33-4fce-bb09-6a8f5b0cc421" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:992eb68a-753e-42b9-803f-104a349481cf", + "resource": { + "resourceType": "Claim", + "id": "992eb68a-753e-42b9-803f-104a349481cf", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1968-02-19T04:15:51-08:00", + "end": "1968-02-19T04:30:51-08:00" + }, + "created": "1968-02-19T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9708a45f-003e-47db-a707-d9f655b08980", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9708a45f-003e-47db-a707-d9f655b08980", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Anthem" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Anthem" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "992eb68a-753e-42b9-803f-104a349481cf" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1968-02-19T04:30:51-08:00", + "end": "1969-02-19T04:30:51-08:00" + }, + "created": "1968-02-19T04:30:51-08:00", + "insurer": { + "display": "Anthem" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:992eb68a-753e-42b9-803f-104a349481cf" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1968-02-19T04:15:51-08:00", + "end": "1968-02-19T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd", + "resource": { + "resourceType": "Encounter", + "id": "68bc47dc-5742-45b9-be21-55f0219512dd", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1969-02-24T04:15:51-08:00", + "end": "1969-02-24T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1969-02-24T04:15:51-08:00", + "end": "1969-02-24T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8c3c0828-d651-4585-ac06-19de5acf9c03", + "resource": { + "resourceType": "MedicationRequest", + "id": "8c3c0828-d651-4585-ac06-19de5acf9c03", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" + }, + "authoredOn": "1969-02-24T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:cb0963df-89fa-4953-b29c-b858683f523f", + "resource": { + "resourceType": "Claim", + "id": "cb0963df-89fa-4953-b29c-b858683f523f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1969-02-24T04:15:51-08:00", + "end": "1969-02-24T04:45:51-08:00" + }, + "created": "1969-02-24T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:8c3c0828-d651-4585-ac06-19de5acf9c03" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4", + "resource": { + "resourceType": "Claim", + "id": "75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1969-02-24T04:15:51-08:00", + "end": "1969-02-24T04:45:51-08:00" + }, + "created": "1969-02-24T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2f6fff0d-12d7-4a29-baca-f92f9cc4ca21", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2f6fff0d-12d7-4a29-baca-f92f9cc4ca21", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Anthem" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Anthem" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1969-02-24T04:45:51-08:00", + "end": "1970-02-24T04:45:51-08:00" + }, + "created": "1969-02-24T04:45:51-08:00", + "insurer": { + "display": "Anthem" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Anthem" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1969-02-24T04:15:51-08:00", + "end": "1969-02-24T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c", + "resource": { + "resourceType": "Encounter", + "id": "b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1970-03-02T04:15:51-08:00", + "end": "1970-03-02T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1970-03-02T04:15:51-08:00", + "end": "1970-03-02T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:45640fcb-44b4-4d63-8137-4a3e244083b2", + "resource": { + "resourceType": "MedicationRequest", + "id": "45640fcb-44b4-4d63-8137-4a3e244083b2", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" + }, + "authoredOn": "1970-03-02T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:6b632566-b7a6-4e62-b64e-6a0408b5c9ff", + "resource": { + "resourceType": "Claim", + "id": "6b632566-b7a6-4e62-b64e-6a0408b5c9ff", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1970-03-02T04:15:51-08:00", + "end": "1970-03-02T04:45:51-08:00" + }, + "created": "1970-03-02T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:45640fcb-44b4-4d63-8137-4a3e244083b2" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ffbb426e-ce1d-4bd6-bd42-a4938cb19d04", + "resource": { + "resourceType": "Claim", + "id": "ffbb426e-ce1d-4bd6-bd42-a4938cb19d04", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1970-03-02T04:15:51-08:00", + "end": "1970-03-02T04:45:51-08:00" + }, + "created": "1970-03-02T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:346a9e8a-bafc-4729-a31d-00670c0933ec", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "346a9e8a-bafc-4729-a31d-00670c0933ec", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ffbb426e-ce1d-4bd6-bd42-a4938cb19d04" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1970-03-02T04:45:51-08:00", + "end": "1971-03-02T04:45:51-08:00" + }, + "created": "1970-03-02T04:45:51-08:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ffbb426e-ce1d-4bd6-bd42-a4938cb19d04" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1970-03-02T04:15:51-08:00", + "end": "1970-03-02T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163", + "resource": { + "resourceType": "Encounter", + "id": "0d03108a-c9c2-4134-99b1-6a58b8d61163", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1971-03-08T04:15:51-08:00", + "end": "1971-03-08T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1971-03-08T04:15:51-08:00", + "end": "1971-03-08T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f68524e7-b47e-4495-95cd-c18426e25643", + "resource": { + "resourceType": "MedicationRequest", + "id": "f68524e7-b47e-4495-95cd-c18426e25643", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" + }, + "authoredOn": "1971-03-08T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:bfb8ea92-2de4-4f19-9bcb-0b6e1080035b", + "resource": { + "resourceType": "Claim", + "id": "bfb8ea92-2de4-4f19-9bcb-0b6e1080035b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1971-03-08T04:15:51-08:00", + "end": "1971-03-08T04:30:51-08:00" + }, + "created": "1971-03-08T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:f68524e7-b47e-4495-95cd-c18426e25643" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:99650392-90a6-4f14-aa47-34fdfe24f274", + "resource": { + "resourceType": "Claim", + "id": "99650392-90a6-4f14-aa47-34fdfe24f274", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1971-03-08T04:15:51-08:00", + "end": "1971-03-08T04:30:51-08:00" + }, + "created": "1971-03-08T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e286c550-3649-46dd-beef-10bf70a33ee3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e286c550-3649-46dd-beef-10bf70a33ee3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "99650392-90a6-4f14-aa47-34fdfe24f274" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1971-03-08T04:30:51-08:00", + "end": "1972-03-08T04:30:51-08:00" + }, + "created": "1971-03-08T04:30:51-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:99650392-90a6-4f14-aa47-34fdfe24f274" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1971-03-08T04:15:51-08:00", + "end": "1971-03-08T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc", + "resource": { + "resourceType": "Encounter", + "id": "ef56877c-c52a-4644-9a93-a2ad9a9f08dc", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1972-03-13T04:15:51-08:00", + "end": "1972-03-13T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1972-03-13T04:15:51-08:00", + "end": "1972-03-13T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bbd477f9-bb7f-4571-af62-f3f133e291e1", + "resource": { + "resourceType": "MedicationRequest", + "id": "bbd477f9-bb7f-4571-af62-f3f133e291e1", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" + }, + "authoredOn": "1972-03-13T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8d68d79a-8a7c-4d8e-81ef-8c118a99ca71", + "resource": { + "resourceType": "Claim", + "id": "8d68d79a-8a7c-4d8e-81ef-8c118a99ca71", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-03-13T04:15:51-08:00", + "end": "1972-03-13T04:45:51-08:00" + }, + "created": "1972-03-13T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:bbd477f9-bb7f-4571-af62-f3f133e291e1" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:73a52d4c-ebd0-4887-8677-e33e3a98d116", + "resource": { + "resourceType": "Claim", + "id": "73a52d4c-ebd0-4887-8677-e33e3a98d116", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1972-03-13T04:15:51-08:00", + "end": "1972-03-13T04:45:51-08:00" + }, + "created": "1972-03-13T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:76a827e3-25b7-4ae4-b249-58b8fb79fb33", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "76a827e3-25b7-4ae4-b249-58b8fb79fb33", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "73a52d4c-ebd0-4887-8677-e33e3a98d116" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-03-13T04:45:51-08:00", + "end": "1973-03-13T04:45:51-08:00" + }, + "created": "1972-03-13T04:45:51-08:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:73a52d4c-ebd0-4887-8677-e33e3a98d116" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1972-03-13T04:15:51-08:00", + "end": "1972-03-13T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7", + "resource": { + "resourceType": "Encounter", + "id": "c47302e4-bb57-4490-a42e-e01c90ccf8f7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1972-09-18T05:15:51-07:00", + "end": "1972-09-18T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1972-09-18T05:15:51-07:00", + "end": "1972-09-18T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a7cfc82f-273e-45cb-a6ea-0ed1320fc709", + "resource": { + "resourceType": "MedicationRequest", + "id": "a7cfc82f-273e-45cb-a6ea-0ed1320fc709", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" + }, + "authoredOn": "1972-09-18T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:af048b54-7375-4aa0-8d13-06670e09747f", + "resource": { + "resourceType": "Claim", + "id": "af048b54-7375-4aa0-8d13-06670e09747f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-09-18T05:15:51-07:00", + "end": "1972-09-18T05:30:51-07:00" + }, + "created": "1972-09-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:a7cfc82f-273e-45cb-a6ea-0ed1320fc709" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c49ee98c-d8b1-4599-8020-d2aaeed2999c", + "resource": { + "resourceType": "Claim", + "id": "c49ee98c-d8b1-4599-8020-d2aaeed2999c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1972-09-18T05:15:51-07:00", + "end": "1972-09-18T05:30:51-07:00" + }, + "created": "1972-09-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5b904601-474f-4285-8282-b0b9e147f594", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5b904601-474f-4285-8282-b0b9e147f594", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c49ee98c-d8b1-4599-8020-d2aaeed2999c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-09-18T05:30:51-07:00", + "end": "1973-09-18T05:30:51-07:00" + }, + "created": "1972-09-18T05:30:51-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:c49ee98c-d8b1-4599-8020-d2aaeed2999c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "1972-09-18T05:15:51-07:00", + "end": "1972-09-18T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8", + "resource": { + "resourceType": "Encounter", + "id": "220a9e60-1006-4228-b2ae-42375684add8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1972-09-13T05:15:51-07:00", + "end": "1972-09-13T05:40:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1972-09-13T05:15:51-07:00", + "end": "1972-09-13T05:40:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a25fd790-0bec-4227-8b14-c0cd89a034f0", + "resource": { + "resourceType": "MedicationRequest", + "id": "a25fd790-0bec-4227-8b14-c0cd89a034f0", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" + }, + "authoredOn": "1972-09-18T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:69eb6011-54c3-49db-b92c-3a2cf10a3273", + "resource": { + "resourceType": "Claim", + "id": "69eb6011-54c3-49db-b92c-3a2cf10a3273", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-09-13T05:15:51-07:00", + "end": "1972-09-13T05:40:51-07:00" + }, + "created": "1972-09-13T05:40:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:a25fd790-0bec-4227-8b14-c0cd89a034f0" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0fac4e42-6633-4d77-9c45-9c83356280d9", + "resource": { + "resourceType": "Claim", + "id": "0fac4e42-6633-4d77-9c45-9c83356280d9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1972-09-13T05:15:51-07:00", + "end": "1972-09-13T05:40:51-07:00" + }, + "created": "1972-09-13T05:40:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:29138263-4c9d-4002-9ee3-b532f5c0d244", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "29138263-4c9d-4002-9ee3-b532f5c0d244", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Cigna Health" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Cigna Health" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0fac4e42-6633-4d77-9c45-9c83356280d9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1972-09-13T05:40:51-07:00", + "end": "1973-09-13T05:40:51-07:00" + }, + "created": "1972-09-13T05:40:51-07:00", + "insurer": { + "display": "Cigna Health" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0fac4e42-6633-4d77-9c45-9c83356280d9" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Cigna Health" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "1972-09-13T05:15:51-07:00", + "end": "1972-09-13T05:40:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379", + "resource": { + "resourceType": "Encounter", + "id": "305d11c5-6d30-49e7-87cf-35dbd794c379", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1973-03-19T04:15:51-08:00", + "end": "1973-03-19T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1973-03-19T04:15:51-08:00", + "end": "1973-03-19T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9", + "resource": { + "resourceType": "MedicationRequest", + "id": "d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" + }, + "authoredOn": "1973-03-19T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:abda0fee-3f27-4461-b3f5-5cd1bdb20605", + "resource": { + "resourceType": "Claim", + "id": "abda0fee-3f27-4461-b3f5-5cd1bdb20605", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1973-03-19T04:15:51-08:00", + "end": "1973-03-19T04:30:51-08:00" + }, + "created": "1973-03-19T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ac30b172-6737-4dbf-8abf-28bad6eb19d3", + "resource": { + "resourceType": "Claim", + "id": "ac30b172-6737-4dbf-8abf-28bad6eb19d3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1973-03-19T04:15:51-08:00", + "end": "1973-03-19T04:30:51-08:00" + }, + "created": "1973-03-19T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e41d7307-5b9b-40ef-867c-a70e20259fab", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e41d7307-5b9b-40ef-867c-a70e20259fab", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ac30b172-6737-4dbf-8abf-28bad6eb19d3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1973-03-19T04:30:51-08:00", + "end": "1974-03-19T04:30:51-07:00" + }, + "created": "1973-03-19T04:30:51-08:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ac30b172-6737-4dbf-8abf-28bad6eb19d3" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1973-03-19T04:15:51-08:00", + "end": "1973-03-19T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed", + "resource": { + "resourceType": "Encounter", + "id": "8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1974-03-25T05:15:51-07:00", + "end": "1974-03-25T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1974-03-25T05:15:51-07:00", + "end": "1974-03-25T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d5557008-cbad-4d0b-8be0-f70608079eac", + "resource": { + "resourceType": "MedicationRequest", + "id": "d5557008-cbad-4d0b-8be0-f70608079eac", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" + }, + "authoredOn": "1974-03-25T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0e214bf1-1d2f-4b69-b2ff-8c9693a41fdb", + "resource": { + "resourceType": "Claim", + "id": "0e214bf1-1d2f-4b69-b2ff-8c9693a41fdb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1974-03-25T05:15:51-07:00", + "end": "1974-03-25T05:45:51-07:00" + }, + "created": "1974-03-25T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:d5557008-cbad-4d0b-8be0-f70608079eac" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:466edc12-b2eb-433c-b1c7-deb51aa2d319", + "resource": { + "resourceType": "Claim", + "id": "466edc12-b2eb-433c-b1c7-deb51aa2d319", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1974-03-25T05:15:51-07:00", + "end": "1974-03-25T05:45:51-07:00" + }, + "created": "1974-03-25T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1ddd9d3a-2a99-4c89-9b95-24081772e1ca", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1ddd9d3a-2a99-4c89-9b95-24081772e1ca", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "466edc12-b2eb-433c-b1c7-deb51aa2d319" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1974-03-25T05:45:51-07:00", + "end": "1975-03-25T05:45:51-07:00" + }, + "created": "1974-03-25T05:45:51-07:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:466edc12-b2eb-433c-b1c7-deb51aa2d319" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1974-03-25T05:15:51-07:00", + "end": "1974-03-25T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66", + "resource": { + "resourceType": "Encounter", + "id": "57fee96f-a86d-4544-afa1-f6f6c8e37d66", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1975-03-31T05:15:51-07:00", + "end": "1975-03-31T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1975-03-31T05:15:51-07:00", + "end": "1975-03-31T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:82eaade0-3a05-4290-9324-9c4c2b0473bf", + "resource": { + "resourceType": "MedicationRequest", + "id": "82eaade0-3a05-4290-9324-9c4c2b0473bf", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" + }, + "authoredOn": "1975-03-31T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:c687d65a-2f8a-46e1-b8b5-ed89d1852087", + "resource": { + "resourceType": "Claim", + "id": "c687d65a-2f8a-46e1-b8b5-ed89d1852087", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1975-03-31T05:15:51-07:00", + "end": "1975-03-31T05:30:51-07:00" + }, + "created": "1975-03-31T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:82eaade0-3a05-4290-9324-9c4c2b0473bf" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e0bb258a-6080-4f6a-b2d3-7fbe8928eace", + "resource": { + "resourceType": "Claim", + "id": "e0bb258a-6080-4f6a-b2d3-7fbe8928eace", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1975-03-31T05:15:51-07:00", + "end": "1975-03-31T05:30:51-07:00" + }, + "created": "1975-03-31T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b152873b-f16d-40a8-92db-f2a3f42607fa", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b152873b-f16d-40a8-92db-f2a3f42607fa", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e0bb258a-6080-4f6a-b2d3-7fbe8928eace" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1975-03-31T05:30:51-07:00", + "end": "1976-03-31T05:30:51-08:00" + }, + "created": "1975-03-31T05:30:51-07:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e0bb258a-6080-4f6a-b2d3-7fbe8928eace" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1975-03-31T05:15:51-07:00", + "end": "1975-03-31T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e", + "resource": { + "resourceType": "Encounter", + "id": "35707937-d9c0-4f0b-824f-8f40abfccf9e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1976-04-05T04:15:51-08:00", + "end": "1976-04-05T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1976-04-05T04:15:51-08:00", + "end": "1976-04-05T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:33efd06d-3665-4930-82d7-5ecd7fc95b20", + "resource": { + "resourceType": "MedicationRequest", + "id": "33efd06d-3665-4930-82d7-5ecd7fc95b20", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" + }, + "authoredOn": "1976-04-05T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5f18df60-3875-4f96-895c-3c7f1e4589b1", + "resource": { + "resourceType": "Claim", + "id": "5f18df60-3875-4f96-895c-3c7f1e4589b1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1976-04-05T04:15:51-08:00", + "end": "1976-04-05T04:45:51-08:00" + }, + "created": "1976-04-05T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:33efd06d-3665-4930-82d7-5ecd7fc95b20" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Blue Cross Blue Shield" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:59c698a6-66c5-4212-a73f-6db2c77a999a", + "resource": { + "resourceType": "Claim", + "id": "59c698a6-66c5-4212-a73f-6db2c77a999a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1976-04-05T04:15:51-08:00", + "end": "1976-04-05T04:45:51-08:00" + }, + "created": "1976-04-05T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Blue Cross Blue Shield" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2651c4a2-d9e0-4aa1-8f0e-d763b1295ea4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2651c4a2-d9e0-4aa1-8f0e-d763b1295ea4", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Blue Cross Blue Shield" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Blue Cross Blue Shield" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "59c698a6-66c5-4212-a73f-6db2c77a999a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1976-04-05T04:45:51-08:00", + "end": "1977-04-05T04:45:51-08:00" + }, + "created": "1976-04-05T04:45:51-08:00", + "insurer": { + "display": "Blue Cross Blue Shield" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:59c698a6-66c5-4212-a73f-6db2c77a999a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Blue Cross Blue Shield" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1976-04-05T04:15:51-08:00", + "end": "1976-04-05T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555", + "resource": { + "resourceType": "Encounter", + "id": "d2d91306-a6ad-442e-a83f-6d97844de555", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1977-04-11T04:15:51-08:00", + "end": "1977-04-11T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1977-04-11T04:15:51-08:00", + "end": "1977-04-11T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:75768d8c-5e38-4a02-9e32-b578a957abe1", + "resource": { + "resourceType": "MedicationRequest", + "id": "75768d8c-5e38-4a02-9e32-b578a957abe1", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" + }, + "authoredOn": "1977-04-11T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8b7edca7-5f49-4fbb-bbaa-76958659f7be", + "resource": { + "resourceType": "Claim", + "id": "8b7edca7-5f49-4fbb-bbaa-76958659f7be", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1977-04-11T04:15:51-08:00", + "end": "1977-04-11T04:30:51-08:00" + }, + "created": "1977-04-11T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:75768d8c-5e38-4a02-9e32-b578a957abe1" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:636ac4e4-059b-4dc9-a38c-66f24f46a47c", + "resource": { + "resourceType": "Claim", + "id": "636ac4e4-059b-4dc9-a38c-66f24f46a47c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1977-04-11T04:15:51-08:00", + "end": "1977-04-11T04:30:51-08:00" + }, + "created": "1977-04-11T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fda88127-d697-4d34-9f77-b5adf3381a17", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "fda88127-d697-4d34-9f77-b5adf3381a17", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "636ac4e4-059b-4dc9-a38c-66f24f46a47c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1977-04-11T04:30:51-08:00", + "end": "1978-04-11T04:30:51-08:00" + }, + "created": "1977-04-11T04:30:51-08:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:636ac4e4-059b-4dc9-a38c-66f24f46a47c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1977-04-11T04:15:51-08:00", + "end": "1977-04-11T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217", + "resource": { + "resourceType": "Encounter", + "id": "1514ab4e-4d9f-4518-8364-e8af6bcc8217", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1978-04-17T04:15:51-08:00", + "end": "1978-04-17T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1978-04-17T04:15:51-08:00", + "end": "1978-04-17T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:db6b3a0f-d791-48cf-8a81-5601a71b2043", + "resource": { + "resourceType": "MedicationRequest", + "id": "db6b3a0f-d791-48cf-8a81-5601a71b2043", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" + }, + "authoredOn": "1978-04-17T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8fa05e2c-fddd-46ff-9357-1e5c02ddb766", + "resource": { + "resourceType": "Claim", + "id": "8fa05e2c-fddd-46ff-9357-1e5c02ddb766", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1978-04-17T04:15:51-08:00", + "end": "1978-04-17T04:45:51-08:00" + }, + "created": "1978-04-17T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:db6b3a0f-d791-48cf-8a81-5601a71b2043" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:36e36508-cae4-461c-bbd0-6bbb7a0a4dea", + "resource": { + "resourceType": "Claim", + "id": "36e36508-cae4-461c-bbd0-6bbb7a0a4dea", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1978-04-17T04:15:51-08:00", + "end": "1978-04-17T04:45:51-08:00" + }, + "created": "1978-04-17T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bbef9ec2-58fb-4ccd-ad38-fb140ebfc3bc", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "bbef9ec2-58fb-4ccd-ad38-fb140ebfc3bc", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "36e36508-cae4-461c-bbd0-6bbb7a0a4dea" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1978-04-17T04:45:51-08:00", + "end": "1979-04-17T04:45:51-08:00" + }, + "created": "1978-04-17T04:45:51-08:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:36e36508-cae4-461c-bbd0-6bbb7a0a4dea" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1978-04-17T04:15:51-08:00", + "end": "1978-04-17T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188", + "resource": { + "resourceType": "Encounter", + "id": "4f930943-2e18-4fc3-9573-cd2d637e0188", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a", + "resource": { + "resourceType": "Condition", + "id": "7e29d4a9-b5b9-4965-bfa6-02b3ad26418a", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + }, + "onsetDateTime": "1979-04-23T04:15:51-08:00", + "recordedDate": "1979-04-23T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ed07769e-89e0-40df-86e9-48dcbbd2e97d", + "resource": { + "resourceType": "MedicationRequest", + "id": "ed07769e-89e0-40df-86e9-48dcbbd2e97d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + }, + "authoredOn": "1979-04-23T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5c8c256c-9428-49c9-bef9-a78f6a64f2ba", + "resource": { + "resourceType": "Claim", + "id": "5c8c256c-9428-49c9-bef9-a78f6a64f2ba", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "created": "1979-04-23T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ed07769e-89e0-40df-86e9-48dcbbd2e97d" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9", + "resource": { + "resourceType": "CareTeam", + "id": "ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9", + "status": "active", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + }, + "period": { + "start": "1979-04-23T04:15:51-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:26c6277e-cc3a-47bd-8838-fba69ceb3a82", + "resource": { + "resourceType": "CarePlan", + "id": "26c6277e-cc3a-47bd-8838-fba69ceb3a82", + "text": { + "status": "generated", + "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698360004", + "display": "Diabetes self management plan" + } + ], + "text": "Diabetes self management plan" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + }, + "period": { + "start": "1979-04-23T04:15:51-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9" + } + ], + "addresses": [ + { + "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "160670007", + "display": "Diabetic diet" + } + ], + "text": "Diabetic diet" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229065009", + "display": "Exercise therapy" + } + ], + "text": "Exercise therapy" + }, + "status": "in-progress", + "location": { + "display": "PCP128586" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:c3eea979-eb01-406b-952f-76483bb31ca0", + "resource": { + "resourceType": "Claim", + "id": "c3eea979-eb01-406b-952f-76483bb31ca0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "created": "1979-04-23T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:32fd4fad-501a-4730-96a1-371ede601291", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "32fd4fad-501a-4730-96a1-371ede601291", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c3eea979-eb01-406b-952f-76483bb31ca0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1979-04-23T04:30:51-08:00", + "end": "1980-04-23T04:30:51-08:00" + }, + "created": "1979-04-23T04:30:51-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:c3eea979-eb01-406b-952f-76483bb31ca0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "15777000", + "display": "Prediabetes" + } + ], + "text": "Prediabetes" + }, + "servicedPeriod": { + "start": "1979-04-23T04:15:51-08:00", + "end": "1979-04-23T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9", + "resource": { + "resourceType": "Encounter", + "id": "d96d0c1a-6842-44fe-8e28-20d437a374c9", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1980-04-28T05:15:51-07:00", + "end": "1980-04-28T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1980-04-28T05:15:51-07:00", + "end": "1980-04-28T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0c479a35-4f68-4b13-9193-e3ae5c98d28a", + "resource": { + "resourceType": "MedicationRequest", + "id": "0c479a35-4f68-4b13-9193-e3ae5c98d28a", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" + }, + "authoredOn": "1980-04-28T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:dfce59ce-4bd0-480e-aed4-fa8b355b4fb8", + "resource": { + "resourceType": "Claim", + "id": "dfce59ce-4bd0-480e-aed4-fa8b355b4fb8", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1980-04-28T05:15:51-07:00", + "end": "1980-04-28T05:30:51-07:00" + }, + "created": "1980-04-28T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0c479a35-4f68-4b13-9193-e3ae5c98d28a" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:62a209ad-10fe-412c-ba4d-b605321c027f", + "resource": { + "resourceType": "Claim", + "id": "62a209ad-10fe-412c-ba4d-b605321c027f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1980-04-28T05:15:51-07:00", + "end": "1980-04-28T05:30:51-07:00" + }, + "created": "1980-04-28T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3c55935b-b90c-4ca4-a9a2-442cf767597b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3c55935b-b90c-4ca4-a9a2-442cf767597b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "62a209ad-10fe-412c-ba4d-b605321c027f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1980-04-28T05:30:51-07:00", + "end": "1981-04-28T05:30:51-07:00" + }, + "created": "1980-04-28T05:30:51-07:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:62a209ad-10fe-412c-ba4d-b605321c027f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1980-04-28T05:15:51-07:00", + "end": "1980-04-28T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39", + "resource": { + "resourceType": "Encounter", + "id": "d172e169-67dd-4311-ac98-c79c0b5a2b39", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1981-05-04T05:15:51-07:00", + "end": "1981-05-04T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1981-05-04T05:15:51-07:00", + "end": "1981-05-04T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:57131a9d-b05a-4844-9981-69d85ae54117", + "resource": { + "resourceType": "MedicationRequest", + "id": "57131a9d-b05a-4844-9981-69d85ae54117", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" + }, + "authoredOn": "1981-05-04T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:af120bbe-ebf6-45f4-ae26-4fd080a91c2c", + "resource": { + "resourceType": "Claim", + "id": "af120bbe-ebf6-45f4-ae26-4fd080a91c2c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1981-05-04T05:15:51-07:00", + "end": "1981-05-04T05:30:51-07:00" + }, + "created": "1981-05-04T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:57131a9d-b05a-4844-9981-69d85ae54117" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:907e442a-2460-465c-9ceb-f4050ac15d9a", + "resource": { + "resourceType": "Claim", + "id": "907e442a-2460-465c-9ceb-f4050ac15d9a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1981-05-04T05:15:51-07:00", + "end": "1981-05-04T05:30:51-07:00" + }, + "created": "1981-05-04T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a62decf1-8628-49df-976e-547142fcc11b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a62decf1-8628-49df-976e-547142fcc11b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicaid" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicaid" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "907e442a-2460-465c-9ceb-f4050ac15d9a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1981-05-04T05:30:51-07:00", + "end": "1982-05-04T05:30:51-07:00" + }, + "created": "1981-05-04T05:30:51-07:00", + "insurer": { + "display": "Medicaid" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:907e442a-2460-465c-9ceb-f4050ac15d9a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicaid" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1981-05-04T05:15:51-07:00", + "end": "1981-05-04T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321", + "resource": { + "resourceType": "Encounter", + "id": "efd7d1b4-a401-4239-8984-002f8ab6c321", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1982-05-10T05:15:51-07:00", + "end": "1982-05-10T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1982-05-10T05:15:51-07:00", + "end": "1982-05-10T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:06f498c9-eb07-499e-b73e-cd95843f9d51", + "resource": { + "resourceType": "MedicationRequest", + "id": "06f498c9-eb07-499e-b73e-cd95843f9d51", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" + }, + "authoredOn": "1982-05-10T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0aa4a8d2-c79e-4a8f-af81-3af95daf630b", + "resource": { + "resourceType": "Claim", + "id": "0aa4a8d2-c79e-4a8f-af81-3af95daf630b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1982-05-10T05:15:51-07:00", + "end": "1982-05-10T05:30:51-07:00" + }, + "created": "1982-05-10T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:06f498c9-eb07-499e-b73e-cd95843f9d51" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5", + "resource": { + "resourceType": "Claim", + "id": "6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1982-05-10T05:15:51-07:00", + "end": "1982-05-10T05:30:51-07:00" + }, + "created": "1982-05-10T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:97e0e4b6-140f-41d6-86c2-16eec511e6fb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "97e0e4b6-140f-41d6-86c2-16eec511e6fb", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1982-05-10T05:30:51-07:00", + "end": "1983-05-10T05:30:51-07:00" + }, + "created": "1982-05-10T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1982-05-10T05:15:51-07:00", + "end": "1982-05-10T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0", + "resource": { + "resourceType": "Encounter", + "id": "083f8d20-2266-40bb-8dad-315bea4de8a0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1983-05-16T05:15:51-07:00", + "end": "1983-05-16T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1983-05-16T05:15:51-07:00", + "end": "1983-05-16T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1f013ce3-f405-4b5f-8572-c6953663f659", + "resource": { + "resourceType": "MedicationRequest", + "id": "1f013ce3-f405-4b5f-8572-c6953663f659", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" + }, + "authoredOn": "1983-05-16T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:3f5e693c-e540-4411-968c-747723cfbe03", + "resource": { + "resourceType": "Claim", + "id": "3f5e693c-e540-4411-968c-747723cfbe03", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1983-05-16T05:15:51-07:00", + "end": "1983-05-16T05:45:51-07:00" + }, + "created": "1983-05-16T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:1f013ce3-f405-4b5f-8572-c6953663f659" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1224c44e-8f7a-4ea6-846c-9adad0fb89c2", + "resource": { + "resourceType": "Claim", + "id": "1224c44e-8f7a-4ea6-846c-9adad0fb89c2", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1983-05-16T05:15:51-07:00", + "end": "1983-05-16T05:45:51-07:00" + }, + "created": "1983-05-16T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:310488a9-7ee4-47dc-9fda-c675b9c47e9c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "310488a9-7ee4-47dc-9fda-c675b9c47e9c", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1224c44e-8f7a-4ea6-846c-9adad0fb89c2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1983-05-16T05:45:51-07:00", + "end": "1984-05-16T05:45:51-07:00" + }, + "created": "1983-05-16T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:1224c44e-8f7a-4ea6-846c-9adad0fb89c2" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1983-05-16T05:15:51-07:00", + "end": "1983-05-16T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e", + "resource": { + "resourceType": "Encounter", + "id": "f2af01b9-93e5-4aeb-b710-86875c752c9e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1984-05-21T05:15:51-07:00", + "end": "1984-05-21T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1984-05-21T05:15:51-07:00", + "end": "1984-05-21T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0ed9f205-be7a-44cb-b103-7ca50e493292", + "resource": { + "resourceType": "MedicationRequest", + "id": "0ed9f205-be7a-44cb-b103-7ca50e493292", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" + }, + "authoredOn": "1984-05-21T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:3e5c63f7-3be6-453d-b5ef-b7ae74c275c0", + "resource": { + "resourceType": "Claim", + "id": "3e5c63f7-3be6-453d-b5ef-b7ae74c275c0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1984-05-21T05:15:51-07:00", + "end": "1984-05-21T05:45:51-07:00" + }, + "created": "1984-05-21T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0ed9f205-be7a-44cb-b103-7ca50e493292" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c", + "resource": { + "resourceType": "Claim", + "id": "0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1984-05-21T05:15:51-07:00", + "end": "1984-05-21T05:45:51-07:00" + }, + "created": "1984-05-21T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d650b7de-fa34-4e57-aded-1bba643c9e04", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d650b7de-fa34-4e57-aded-1bba643c9e04", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1984-05-21T05:45:51-07:00", + "end": "1985-05-21T05:45:51-07:00" + }, + "created": "1984-05-21T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1984-05-21T05:15:51-07:00", + "end": "1984-05-21T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e", + "resource": { + "resourceType": "Encounter", + "id": "6be1a524-887d-4221-8114-df00cdd6e68e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1985-05-27T05:15:51-07:00", + "end": "1985-05-27T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1985-05-27T05:15:51-07:00", + "end": "1985-05-27T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1a1dba2d-3cf7-4391-b54d-c05eff05910b", + "resource": { + "resourceType": "MedicationRequest", + "id": "1a1dba2d-3cf7-4391-b54d-c05eff05910b", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" + }, + "authoredOn": "1985-05-27T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:9232907d-e3f1-452c-ac47-c2f93ac25580", + "resource": { + "resourceType": "Claim", + "id": "9232907d-e3f1-452c-ac47-c2f93ac25580", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1985-05-27T05:15:51-07:00", + "end": "1985-05-27T05:30:51-07:00" + }, + "created": "1985-05-27T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:1a1dba2d-3cf7-4391-b54d-c05eff05910b" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4013d214-b869-445e-9f37-81e756896650", + "resource": { + "resourceType": "Claim", + "id": "4013d214-b869-445e-9f37-81e756896650", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1985-05-27T05:15:51-07:00", + "end": "1985-05-27T05:30:51-07:00" + }, + "created": "1985-05-27T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:605c40b9-5636-4f69-9c46-637cbdf1ca14", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "605c40b9-5636-4f69-9c46-637cbdf1ca14", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4013d214-b869-445e-9f37-81e756896650" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1985-05-27T05:30:51-07:00", + "end": "1986-05-27T05:30:51-07:00" + }, + "created": "1985-05-27T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4013d214-b869-445e-9f37-81e756896650" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1985-05-27T05:15:51-07:00", + "end": "1985-05-27T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf", + "resource": { + "resourceType": "Encounter", + "id": "18acea1a-e5a0-4167-9e6b-b957ceae92cf", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1986-06-02T05:15:51-07:00", + "end": "1986-06-02T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1986-06-02T05:15:51-07:00", + "end": "1986-06-02T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:42fbc707-db93-4efb-bfb9-4d97192d898f", + "resource": { + "resourceType": "MedicationRequest", + "id": "42fbc707-db93-4efb-bfb9-4d97192d898f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" + }, + "authoredOn": "1986-06-02T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d67ee42b-c853-44bb-a039-ead835f1b47b", + "resource": { + "resourceType": "Claim", + "id": "d67ee42b-c853-44bb-a039-ead835f1b47b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1986-06-02T05:15:51-07:00", + "end": "1986-06-02T05:45:51-07:00" + }, + "created": "1986-06-02T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:42fbc707-db93-4efb-bfb9-4d97192d898f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8343277b-1210-4e65-9b64-a5a56a413363", + "resource": { + "resourceType": "Claim", + "id": "8343277b-1210-4e65-9b64-a5a56a413363", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1986-06-02T05:15:51-07:00", + "end": "1986-06-02T05:45:51-07:00" + }, + "created": "1986-06-02T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3ea0421f-f3a3-4293-b82a-254247dc8079", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3ea0421f-f3a3-4293-b82a-254247dc8079", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8343277b-1210-4e65-9b64-a5a56a413363" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1986-06-02T05:45:51-07:00", + "end": "1987-06-02T05:45:51-07:00" + }, + "created": "1986-06-02T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8343277b-1210-4e65-9b64-a5a56a413363" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1986-06-02T05:15:51-07:00", + "end": "1986-06-02T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909", + "resource": { + "resourceType": "Encounter", + "id": "f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1987-06-08T05:15:51-07:00", + "end": "1987-06-08T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1987-06-08T05:15:51-07:00", + "end": "1987-06-08T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:dc0523b1-d74a-4472-9efb-3dfa96584c29", + "resource": { + "resourceType": "MedicationRequest", + "id": "dc0523b1-d74a-4472-9efb-3dfa96584c29", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" + }, + "authoredOn": "1987-06-08T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a0a88256-2b5d-41a5-b14d-a1f55dd3a22b", + "resource": { + "resourceType": "Claim", + "id": "a0a88256-2b5d-41a5-b14d-a1f55dd3a22b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-06-08T05:15:51-07:00", + "end": "1987-06-08T05:45:51-07:00" + }, + "created": "1987-06-08T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:dc0523b1-d74a-4472-9efb-3dfa96584c29" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:99f0bc94-4c7d-43b2-aa7d-a10111086f4c", + "resource": { + "resourceType": "Claim", + "id": "99f0bc94-4c7d-43b2-aa7d-a10111086f4c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1987-06-08T05:15:51-07:00", + "end": "1987-06-08T05:45:51-07:00" + }, + "created": "1987-06-08T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6e2c600e-ebf0-46a1-ae92-732bb31be0af", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6e2c600e-ebf0-46a1-ae92-732bb31be0af", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "99f0bc94-4c7d-43b2-aa7d-a10111086f4c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-06-08T05:45:51-07:00", + "end": "1988-06-08T05:45:51-07:00" + }, + "created": "1987-06-08T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:99f0bc94-4c7d-43b2-aa7d-a10111086f4c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1987-06-08T05:15:51-07:00", + "end": "1987-06-08T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413", + "resource": { + "resourceType": "Encounter", + "id": "676b4f93-6ea2-40f2-9c74-c8cd9f7ff413", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1987-08-10T05:15:51-07:00", + "end": "1987-08-10T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1987-08-10T05:15:51-07:00", + "end": "1987-08-10T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:941f3a51-4fce-4310-9cbc-543c82a65a1f", + "resource": { + "resourceType": "MedicationRequest", + "id": "941f3a51-4fce-4310-9cbc-543c82a65a1f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" + }, + "authoredOn": "1987-08-10T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:bc8e4b0c-0570-4ffc-b5f9-0a5f37174d3b", + "resource": { + "resourceType": "Claim", + "id": "bc8e4b0c-0570-4ffc-b5f9-0a5f37174d3b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-08-10T05:15:51-07:00", + "end": "1987-08-10T05:30:51-07:00" + }, + "created": "1987-08-10T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:941f3a51-4fce-4310-9cbc-543c82a65a1f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fbfa1ff7-f922-4174-866f-1b3c40ef041e", + "resource": { + "resourceType": "Claim", + "id": "fbfa1ff7-f922-4174-866f-1b3c40ef041e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1987-08-10T05:15:51-07:00", + "end": "1987-08-10T05:30:51-07:00" + }, + "created": "1987-08-10T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:aa63e71f-322c-4452-8c52-1f50fa3da177", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "aa63e71f-322c-4452-8c52-1f50fa3da177", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fbfa1ff7-f922-4174-866f-1b3c40ef041e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-08-10T05:30:51-07:00", + "end": "1988-08-10T05:30:51-07:00" + }, + "created": "1987-08-10T05:30:51-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:fbfa1ff7-f922-4174-866f-1b3c40ef041e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "1987-08-10T05:15:51-07:00", + "end": "1987-08-10T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e", + "resource": { + "resourceType": "Encounter", + "id": "0668f0dc-d937-4ddc-a810-ec5a8a0a541e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1987-08-05T05:15:51-07:00", + "end": "1987-08-05T05:42:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1987-08-05T05:15:51-07:00", + "end": "1987-08-05T05:42:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5ce622a3-32f9-442a-a25c-6fc4a4f5abc3", + "resource": { + "resourceType": "MedicationRequest", + "id": "5ce622a3-32f9-442a-a25c-6fc4a4f5abc3", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" + }, + "authoredOn": "1987-08-10T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:b187bdaf-b771-48e5-835d-99581ed7895a", + "resource": { + "resourceType": "Claim", + "id": "b187bdaf-b771-48e5-835d-99581ed7895a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-08-05T05:15:51-07:00", + "end": "1987-08-05T05:42:51-07:00" + }, + "created": "1987-08-05T05:42:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:5ce622a3-32f9-442a-a25c-6fc4a4f5abc3" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:78057408-8303-4cec-8355-962090104b54", + "resource": { + "resourceType": "Claim", + "id": "78057408-8303-4cec-8355-962090104b54", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1987-08-05T05:15:51-07:00", + "end": "1987-08-05T05:42:51-07:00" + }, + "created": "1987-08-05T05:42:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:103a5de2-f13a-4b30-b2d2-d509f9b0ce86", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "103a5de2-f13a-4b30-b2d2-d509f9b0ce86", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "78057408-8303-4cec-8355-962090104b54" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1987-08-05T05:42:51-07:00", + "end": "1988-08-05T05:42:51-07:00" + }, + "created": "1987-08-05T05:42:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:78057408-8303-4cec-8355-962090104b54" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "1987-08-05T05:15:51-07:00", + "end": "1987-08-05T05:42:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1", + "resource": { + "resourceType": "Encounter", + "id": "b60aad93-4868-4310-84ce-2c2a8b4885e1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1988-06-13T05:15:51-07:00", + "end": "1988-06-13T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1988-06-13T05:15:51-07:00", + "end": "1988-06-13T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bd1b16d1-af46-4223-9b6f-04dc012badc4", + "resource": { + "resourceType": "MedicationRequest", + "id": "bd1b16d1-af46-4223-9b6f-04dc012badc4", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" + }, + "authoredOn": "1988-06-13T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d50ae18a-e17c-439e-a5fa-9f81b3c7d396", + "resource": { + "resourceType": "Claim", + "id": "d50ae18a-e17c-439e-a5fa-9f81b3c7d396", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1988-06-13T05:15:51-07:00", + "end": "1988-06-13T05:30:51-07:00" + }, + "created": "1988-06-13T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:bd1b16d1-af46-4223-9b6f-04dc012badc4" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f4023f6d-78e8-451c-98a1-adfd07f2d8e5", + "resource": { + "resourceType": "Claim", + "id": "f4023f6d-78e8-451c-98a1-adfd07f2d8e5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1988-06-13T05:15:51-07:00", + "end": "1988-06-13T05:30:51-07:00" + }, + "created": "1988-06-13T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d8b0ab77-9b31-4d93-b60a-cc45da2b15ce", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d8b0ab77-9b31-4d93-b60a-cc45da2b15ce", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f4023f6d-78e8-451c-98a1-adfd07f2d8e5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1988-06-13T05:30:51-07:00", + "end": "1989-06-13T05:30:51-07:00" + }, + "created": "1988-06-13T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:f4023f6d-78e8-451c-98a1-adfd07f2d8e5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1988-06-13T05:15:51-07:00", + "end": "1988-06-13T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784", + "resource": { + "resourceType": "Encounter", + "id": "2e2dea08-86ab-4414-8776-4c1c2878c784", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1989-06-19T05:15:51-07:00", + "end": "1989-06-19T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1989-06-19T05:15:51-07:00", + "end": "1989-06-19T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fa5a1c27-248b-4f16-8abf-5b902ebd14b2", + "resource": { + "resourceType": "MedicationRequest", + "id": "fa5a1c27-248b-4f16-8abf-5b902ebd14b2", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" + }, + "authoredOn": "1989-06-19T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ed4dde50-a7e0-40e3-8acc-0ea0f3bf144c", + "resource": { + "resourceType": "Claim", + "id": "ed4dde50-a7e0-40e3-8acc-0ea0f3bf144c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1989-06-19T05:15:51-07:00", + "end": "1989-06-19T05:45:51-07:00" + }, + "created": "1989-06-19T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:fa5a1c27-248b-4f16-8abf-5b902ebd14b2" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9553833f-ca98-4126-8b52-ae921a23934e", + "resource": { + "resourceType": "Claim", + "id": "9553833f-ca98-4126-8b52-ae921a23934e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1989-06-19T05:15:51-07:00", + "end": "1989-06-19T05:45:51-07:00" + }, + "created": "1989-06-19T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:be6e8fc1-ab29-4dab-b93d-eebf62ccd9b5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "be6e8fc1-ab29-4dab-b93d-eebf62ccd9b5", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9553833f-ca98-4126-8b52-ae921a23934e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1989-06-19T05:45:51-07:00", + "end": "1990-06-19T05:45:51-07:00" + }, + "created": "1989-06-19T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9553833f-ca98-4126-8b52-ae921a23934e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1989-06-19T05:15:51-07:00", + "end": "1989-06-19T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1", + "resource": { + "resourceType": "Encounter", + "id": "a60e4643-1275-4309-8d63-655275b334a1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1990-06-25T05:15:51-07:00", + "end": "1990-06-25T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1990-06-25T05:15:51-07:00", + "end": "1990-06-25T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1eff9cf5-43e3-4b36-9a92-43f50a5da3e6", + "resource": { + "resourceType": "MedicationRequest", + "id": "1eff9cf5-43e3-4b36-9a92-43f50a5da3e6", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" + }, + "authoredOn": "1990-06-25T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:b31486c8-2380-44f0-9d69-ab2f8cbf4784", + "resource": { + "resourceType": "Claim", + "id": "b31486c8-2380-44f0-9d69-ab2f8cbf4784", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1990-06-25T05:15:51-07:00", + "end": "1990-06-25T05:30:51-07:00" + }, + "created": "1990-06-25T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:1eff9cf5-43e3-4b36-9a92-43f50a5da3e6" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a24e410b-ff75-4db9-be01-d2b5cf9a6988", + "resource": { + "resourceType": "Claim", + "id": "a24e410b-ff75-4db9-be01-d2b5cf9a6988", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1990-06-25T05:15:51-07:00", + "end": "1990-06-25T05:30:51-07:00" + }, + "created": "1990-06-25T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0809360b-aeae-45f0-ae10-4487c9de961a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0809360b-aeae-45f0-ae10-4487c9de961a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a24e410b-ff75-4db9-be01-d2b5cf9a6988" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1990-06-25T05:30:51-07:00", + "end": "1991-06-25T05:30:51-07:00" + }, + "created": "1990-06-25T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a24e410b-ff75-4db9-be01-d2b5cf9a6988" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1990-06-25T05:15:51-07:00", + "end": "1990-06-25T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0", + "resource": { + "resourceType": "Encounter", + "id": "5c8d14a8-fb3b-472e-a1fd-065e3b7043f0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1991-06-15T05:15:51-07:00", + "end": "1991-06-15T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1991-06-15T05:15:51-07:00", + "end": "1991-06-15T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "75498004", + "display": "Acute bacterial sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c", + "resource": { + "resourceType": "Condition", + "id": "8a42cdb0-8952-4a22-b347-006f66bb1e0c", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "40055000", + "display": "Chronic sinusitis (disorder)" + } + ], + "text": "Chronic sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" + }, + "onsetDateTime": "1991-06-15T05:15:51-07:00", + "recordedDate": "1991-06-15T05:15:51-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ebb23fe1-d0b8-43df-bace-f2611ec93f1b", + "resource": { + "resourceType": "Claim", + "id": "ebb23fe1-d0b8-43df-bace-f2611ec93f1b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1991-06-15T05:15:51-07:00", + "end": "1991-06-15T05:30:51-07:00" + }, + "created": "1991-06-15T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "40055000", + "display": "Chronic sinusitis (disorder)" + } + ], + "text": "Chronic sinusitis (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2b5aade7-5c7f-41b7-9400-c5a3363f382f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2b5aade7-5c7f-41b7-9400-c5a3363f382f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ebb23fe1-d0b8-43df-bace-f2611ec93f1b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1991-06-15T05:30:51-07:00", + "end": "1992-06-15T05:30:51-07:00" + }, + "created": "1991-06-15T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ebb23fe1-d0b8-43df-bace-f2611ec93f1b" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "1991-06-15T05:15:51-07:00", + "end": "1991-06-15T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "40055000", + "display": "Chronic sinusitis (disorder)" + } + ], + "text": "Chronic sinusitis (disorder)" + }, + "servicedPeriod": { + "start": "1991-06-15T05:15:51-07:00", + "end": "1991-06-15T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932", + "resource": { + "resourceType": "Encounter", + "id": "7ec7b383-ca2b-4d09-a1b1-c56615917932", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1991-06-24T05:15:51-07:00", + "end": "1991-06-24T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "1991-06-24T05:15:51-07:00", + "end": "1991-06-24T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:db639544-126f-4527-94cf-b5b84de4bea9", + "resource": { + "resourceType": "MedicationRequest", + "id": "db639544-126f-4527-94cf-b5b84de4bea9", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" + }, + "authoredOn": "1991-06-24T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1043af7a-95b3-4f63-b5ff-e85e61f29b98", + "resource": { + "resourceType": "Claim", + "id": "1043af7a-95b3-4f63-b5ff-e85e61f29b98", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1991-06-24T05:15:51-07:00", + "end": "1991-06-24T05:30:51-07:00" + }, + "created": "1991-06-24T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:db639544-126f-4527-94cf-b5b84de4bea9" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:10e409dd-2039-4b62-a408-25a8d5d7b245", + "resource": { + "resourceType": "Claim", + "id": "10e409dd-2039-4b62-a408-25a8d5d7b245", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1991-06-24T05:15:51-07:00", + "end": "1991-06-24T05:30:51-07:00" + }, + "created": "1991-06-24T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ce9fa012-bad9-409b-a82f-904b921046f3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ce9fa012-bad9-409b-a82f-904b921046f3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "10e409dd-2039-4b62-a408-25a8d5d7b245" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1991-06-24T05:30:51-07:00", + "end": "1992-06-24T05:30:51-07:00" + }, + "created": "1991-06-24T05:30:51-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:10e409dd-2039-4b62-a408-25a8d5d7b245" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "1991-06-24T05:15:51-07:00", + "end": "1991-06-24T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350", + "resource": { + "resourceType": "Encounter", + "id": "45a4ff1f-b1ae-4d3a-8c17-4a162967a350", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1991-07-01T05:15:51-07:00", + "end": "1991-07-01T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1991-07-01T05:15:51-07:00", + "end": "1991-07-01T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:62f16944-78e7-49e6-8cba-a1063c5567e7", + "resource": { + "resourceType": "MedicationRequest", + "id": "62f16944-78e7-49e6-8cba-a1063c5567e7", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" + }, + "authoredOn": "1991-07-01T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:af037f85-2bb8-40e5-80d5-20ae936d61db", + "resource": { + "resourceType": "Claim", + "id": "af037f85-2bb8-40e5-80d5-20ae936d61db", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1991-07-01T05:15:51-07:00", + "end": "1991-07-01T05:45:51-07:00" + }, + "created": "1991-07-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:62f16944-78e7-49e6-8cba-a1063c5567e7" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b2100338-aadd-49d8-8f7b-6495fc062d0a", + "resource": { + "resourceType": "Claim", + "id": "b2100338-aadd-49d8-8f7b-6495fc062d0a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1991-07-01T05:15:51-07:00", + "end": "1991-07-01T05:45:51-07:00" + }, + "created": "1991-07-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6e35c6f1-a888-4d35-8eb0-3d477b257766", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6e35c6f1-a888-4d35-8eb0-3d477b257766", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b2100338-aadd-49d8-8f7b-6495fc062d0a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1991-07-01T05:45:51-07:00", + "end": "1992-07-01T05:45:51-07:00" + }, + "created": "1991-07-01T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b2100338-aadd-49d8-8f7b-6495fc062d0a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1991-07-01T05:15:51-07:00", + "end": "1991-07-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be", + "resource": { + "resourceType": "Encounter", + "id": "30e7ddad-820e-4e39-b748-8426db7346be", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1992-07-06T05:15:51-07:00", + "end": "1992-07-06T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1992-07-06T05:15:51-07:00", + "end": "1992-07-06T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ed645b29-f27d-4b5a-8f75-88539e2760da", + "resource": { + "resourceType": "MedicationRequest", + "id": "ed645b29-f27d-4b5a-8f75-88539e2760da", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" + }, + "authoredOn": "1992-07-06T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:deb8feb2-58b0-43c3-9da1-653ad057373a", + "resource": { + "resourceType": "Claim", + "id": "deb8feb2-58b0-43c3-9da1-653ad057373a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1992-07-06T05:15:51-07:00", + "end": "1992-07-06T05:45:51-07:00" + }, + "created": "1992-07-06T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ed645b29-f27d-4b5a-8f75-88539e2760da" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8dde5a5f-cf94-4bb6-9ace-f635f1a977f1", + "resource": { + "resourceType": "Claim", + "id": "8dde5a5f-cf94-4bb6-9ace-f635f1a977f1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1992-07-06T05:15:51-07:00", + "end": "1992-07-06T05:45:51-07:00" + }, + "created": "1992-07-06T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:47681d39-1c4e-4cc1-9617-08cc31fd97b1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "47681d39-1c4e-4cc1-9617-08cc31fd97b1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8dde5a5f-cf94-4bb6-9ace-f635f1a977f1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1992-07-06T05:45:51-07:00", + "end": "1993-07-06T05:45:51-07:00" + }, + "created": "1992-07-06T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8dde5a5f-cf94-4bb6-9ace-f635f1a977f1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1992-07-06T05:15:51-07:00", + "end": "1992-07-06T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d", + "resource": { + "resourceType": "Encounter", + "id": "0250c11d-b2e5-4707-8833-3ad4e2946f4d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1993-07-12T05:15:51-07:00", + "end": "1993-07-12T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1993-07-12T05:15:51-07:00", + "end": "1993-07-12T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b1add7db-1732-4fba-8b8c-ea854269c77e", + "resource": { + "resourceType": "MedicationRequest", + "id": "b1add7db-1732-4fba-8b8c-ea854269c77e", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" + }, + "authoredOn": "1993-07-12T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8acc7c2f-dcce-47c1-84cb-3eefe96a1a67", + "resource": { + "resourceType": "Claim", + "id": "8acc7c2f-dcce-47c1-84cb-3eefe96a1a67", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1993-07-12T05:15:51-07:00", + "end": "1993-07-12T05:30:51-07:00" + }, + "created": "1993-07-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:b1add7db-1732-4fba-8b8c-ea854269c77e" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:55458916-e5bb-43f9-9893-f613888efc51", + "resource": { + "resourceType": "Claim", + "id": "55458916-e5bb-43f9-9893-f613888efc51", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1993-07-12T05:15:51-07:00", + "end": "1993-07-12T05:30:51-07:00" + }, + "created": "1993-07-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cded643f-47f5-40b7-ab8f-931c3af87a12", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cded643f-47f5-40b7-ab8f-931c3af87a12", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "55458916-e5bb-43f9-9893-f613888efc51" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1993-07-12T05:30:51-07:00", + "end": "1994-07-12T05:30:51-07:00" + }, + "created": "1993-07-12T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:55458916-e5bb-43f9-9893-f613888efc51" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1993-07-12T05:15:51-07:00", + "end": "1993-07-12T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0", + "resource": { + "resourceType": "Encounter", + "id": "64a03bca-7523-444a-8315-7d6adddb1bc0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1994-07-18T05:15:51-07:00", + "end": "1994-07-18T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1994-07-18T05:15:51-07:00", + "end": "1994-07-18T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bee3c907-561b-4cf2-8678-0fab5469d755", + "resource": { + "resourceType": "MedicationRequest", + "id": "bee3c907-561b-4cf2-8678-0fab5469d755", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" + }, + "authoredOn": "1994-07-18T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:25496d8c-3e18-4176-9280-450d0b12e22e", + "resource": { + "resourceType": "Claim", + "id": "25496d8c-3e18-4176-9280-450d0b12e22e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1994-07-18T05:15:51-07:00", + "end": "1994-07-18T05:30:51-07:00" + }, + "created": "1994-07-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:bee3c907-561b-4cf2-8678-0fab5469d755" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:017eb07c-04ed-461d-8646-afdfa3f2c22c", + "resource": { + "resourceType": "Claim", + "id": "017eb07c-04ed-461d-8646-afdfa3f2c22c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1994-07-18T05:15:51-07:00", + "end": "1994-07-18T05:30:51-07:00" + }, + "created": "1994-07-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f2d5594d-89d8-4758-8403-4d29a157da54", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f2d5594d-89d8-4758-8403-4d29a157da54", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "017eb07c-04ed-461d-8646-afdfa3f2c22c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1994-07-18T05:30:51-07:00", + "end": "1995-07-18T05:30:51-07:00" + }, + "created": "1994-07-18T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:017eb07c-04ed-461d-8646-afdfa3f2c22c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1994-07-18T05:15:51-07:00", + "end": "1994-07-18T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca", + "resource": { + "resourceType": "Encounter", + "id": "a663c449-b5ad-4bc9-b667-117b421f0eca", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1995-07-24T05:15:51-07:00", + "end": "1995-07-24T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1995-07-24T05:15:51-07:00", + "end": "1995-07-24T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6e8d335c-6fe5-4823-9aad-8d7ec0718ea3", + "resource": { + "resourceType": "MedicationRequest", + "id": "6e8d335c-6fe5-4823-9aad-8d7ec0718ea3", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" + }, + "authoredOn": "1995-07-24T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:80c4765c-75fd-44ee-a345-6d802a601eb8", + "resource": { + "resourceType": "Claim", + "id": "80c4765c-75fd-44ee-a345-6d802a601eb8", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1995-07-24T05:15:51-07:00", + "end": "1995-07-24T05:45:51-07:00" + }, + "created": "1995-07-24T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:6e8d335c-6fe5-4823-9aad-8d7ec0718ea3" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2d36dc4b-f1d9-49b4-8ccc-884717e20688", + "resource": { + "resourceType": "Claim", + "id": "2d36dc4b-f1d9-49b4-8ccc-884717e20688", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1995-07-24T05:15:51-07:00", + "end": "1995-07-24T05:45:51-07:00" + }, + "created": "1995-07-24T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:84810259-ab17-4b27-8464-ed2ad8a2bc83", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "84810259-ab17-4b27-8464-ed2ad8a2bc83", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2d36dc4b-f1d9-49b4-8ccc-884717e20688" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1995-07-24T05:45:51-07:00", + "end": "1996-07-24T05:45:51-07:00" + }, + "created": "1995-07-24T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2d36dc4b-f1d9-49b4-8ccc-884717e20688" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1995-07-24T05:15:51-07:00", + "end": "1995-07-24T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f", + "resource": { + "resourceType": "Encounter", + "id": "7c1cbc44-fed4-46c1-9ea8-236b1119153f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1996-07-29T05:15:51-07:00", + "end": "1996-07-29T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1996-07-29T05:15:51-07:00", + "end": "1996-07-29T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:67bcca74-181d-43e4-bd88-8528123ef198", + "resource": { + "resourceType": "MedicationRequest", + "id": "67bcca74-181d-43e4-bd88-8528123ef198", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" + }, + "authoredOn": "1996-07-29T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:634fd458-9041-47f8-a9a0-2bb905940f2e", + "resource": { + "resourceType": "Claim", + "id": "634fd458-9041-47f8-a9a0-2bb905940f2e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1996-07-29T05:15:51-07:00", + "end": "1996-07-29T05:45:51-07:00" + }, + "created": "1996-07-29T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:67bcca74-181d-43e4-bd88-8528123ef198" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:073d30d3-ad05-492c-9dc2-09e878dd1b04", + "resource": { + "resourceType": "Claim", + "id": "073d30d3-ad05-492c-9dc2-09e878dd1b04", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1996-07-29T05:15:51-07:00", + "end": "1996-07-29T05:45:51-07:00" + }, + "created": "1996-07-29T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:31f1002f-2e6c-4367-8736-8685cd95d461", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "31f1002f-2e6c-4367-8736-8685cd95d461", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "073d30d3-ad05-492c-9dc2-09e878dd1b04" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1996-07-29T05:45:51-07:00", + "end": "1997-07-29T05:45:51-07:00" + }, + "created": "1996-07-29T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:073d30d3-ad05-492c-9dc2-09e878dd1b04" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1996-07-29T05:15:51-07:00", + "end": "1996-07-29T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452", + "resource": { + "resourceType": "Encounter", + "id": "639840e0-52f5-4aae-9cc1-c35cbb30d452", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1997-08-04T05:15:51-07:00", + "end": "1997-08-04T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1997-08-04T05:15:51-07:00", + "end": "1997-08-04T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:79ea2871-e47b-4778-8035-c3c21c457976", + "resource": { + "resourceType": "MedicationRequest", + "id": "79ea2871-e47b-4778-8035-c3c21c457976", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" + }, + "authoredOn": "1997-08-04T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0db36527-224d-484f-85e4-0701c4939844", + "resource": { + "resourceType": "Claim", + "id": "0db36527-224d-484f-85e4-0701c4939844", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1997-08-04T05:15:51-07:00", + "end": "1997-08-04T05:45:51-07:00" + }, + "created": "1997-08-04T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:79ea2871-e47b-4778-8035-c3c21c457976" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9b293966-61cf-459b-b77e-40340c1a1f28", + "resource": { + "resourceType": "Claim", + "id": "9b293966-61cf-459b-b77e-40340c1a1f28", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1997-08-04T05:15:51-07:00", + "end": "1997-08-04T05:45:51-07:00" + }, + "created": "1997-08-04T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7d49fe77-98c8-4ba2-bfd8-9feb662bdb1c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7d49fe77-98c8-4ba2-bfd8-9feb662bdb1c", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9b293966-61cf-459b-b77e-40340c1a1f28" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1997-08-04T05:45:51-07:00", + "end": "1998-08-04T05:45:51-07:00" + }, + "created": "1997-08-04T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9b293966-61cf-459b-b77e-40340c1a1f28" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1997-08-04T05:15:51-07:00", + "end": "1997-08-04T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740", + "resource": { + "resourceType": "Encounter", + "id": "a8466f85-3a1a-4247-9e88-80e3f9912740", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1998-08-10T05:15:51-07:00", + "end": "1998-08-10T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1998-08-10T05:15:51-07:00", + "end": "1998-08-10T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ae6181f9-10bd-4ab4-8796-98de69c543da", + "resource": { + "resourceType": "MedicationRequest", + "id": "ae6181f9-10bd-4ab4-8796-98de69c543da", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" + }, + "authoredOn": "1998-08-10T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d69ac9d4-885b-4205-8ffb-aac4bdf01241", + "resource": { + "resourceType": "Claim", + "id": "d69ac9d4-885b-4205-8ffb-aac4bdf01241", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1998-08-10T05:15:51-07:00", + "end": "1998-08-10T05:45:51-07:00" + }, + "created": "1998-08-10T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ae6181f9-10bd-4ab4-8796-98de69c543da" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ca66cb47-3921-4d9f-ac0b-b2f4473f27cb", + "resource": { + "resourceType": "Claim", + "id": "ca66cb47-3921-4d9f-ac0b-b2f4473f27cb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1998-08-10T05:15:51-07:00", + "end": "1998-08-10T05:45:51-07:00" + }, + "created": "1998-08-10T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f47609a2-10b5-496f-a0a8-0a9ccf3729a8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f47609a2-10b5-496f-a0a8-0a9ccf3729a8", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ca66cb47-3921-4d9f-ac0b-b2f4473f27cb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1998-08-10T05:45:51-07:00", + "end": "1999-08-10T05:45:51-07:00" + }, + "created": "1998-08-10T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ca66cb47-3921-4d9f-ac0b-b2f4473f27cb" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1998-08-10T05:15:51-07:00", + "end": "1998-08-10T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5", + "resource": { + "resourceType": "Encounter", + "id": "6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1999-08-16T05:15:51-07:00", + "end": "1999-08-16T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "1999-08-16T05:15:51-07:00", + "end": "1999-08-16T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0cd60bac-b1dc-4dd6-9328-d3cf98976665", + "resource": { + "resourceType": "MedicationRequest", + "id": "0cd60bac-b1dc-4dd6-9328-d3cf98976665", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" + }, + "authoredOn": "1999-08-16T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d062ac5f-6f95-4212-9936-cde99cc84342", + "resource": { + "resourceType": "Claim", + "id": "d062ac5f-6f95-4212-9936-cde99cc84342", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1999-08-16T05:15:51-07:00", + "end": "1999-08-16T05:30:51-07:00" + }, + "created": "1999-08-16T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0cd60bac-b1dc-4dd6-9328-d3cf98976665" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:32e036b7-6ade-46d7-93a7-aa83cbf7811c", + "resource": { + "resourceType": "Claim", + "id": "32e036b7-6ade-46d7-93a7-aa83cbf7811c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "1999-08-16T05:15:51-07:00", + "end": "1999-08-16T05:30:51-07:00" + }, + "created": "1999-08-16T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:024a2154-971d-4861-b4c1-9ac5f54479d0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "024a2154-971d-4861-b4c1-9ac5f54479d0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "32e036b7-6ade-46d7-93a7-aa83cbf7811c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "1999-08-16T05:30:51-07:00", + "end": "2000-08-16T05:30:51-07:00" + }, + "created": "1999-08-16T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:32e036b7-6ade-46d7-93a7-aa83cbf7811c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "1999-08-16T05:15:51-07:00", + "end": "1999-08-16T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c", + "resource": { + "resourceType": "Encounter", + "id": "15cb804f-05bf-4507-a470-fff28bc7f82c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2000-08-21T05:15:51-07:00", + "end": "2000-08-21T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2000-08-21T05:15:51-07:00", + "end": "2000-08-21T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:491b8f1e-db27-4025-9a5a-cd6a4d39de43", + "resource": { + "resourceType": "MedicationRequest", + "id": "491b8f1e-db27-4025-9a5a-cd6a4d39de43", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" + }, + "authoredOn": "2000-08-21T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:6fa61582-e619-4e62-948d-2d81de09332d", + "resource": { + "resourceType": "Claim", + "id": "6fa61582-e619-4e62-948d-2d81de09332d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2000-08-21T05:15:51-07:00", + "end": "2000-08-21T05:30:51-07:00" + }, + "created": "2000-08-21T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:491b8f1e-db27-4025-9a5a-cd6a4d39de43" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a2f13fd6-c438-41e6-bab8-6b1a4663f454", + "resource": { + "resourceType": "Claim", + "id": "a2f13fd6-c438-41e6-bab8-6b1a4663f454", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2000-08-21T05:15:51-07:00", + "end": "2000-08-21T05:30:51-07:00" + }, + "created": "2000-08-21T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:27507818-5e13-4058-91b6-e12fd9e7a556", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "27507818-5e13-4058-91b6-e12fd9e7a556", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a2f13fd6-c438-41e6-bab8-6b1a4663f454" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2000-08-21T05:30:51-07:00", + "end": "2001-08-21T05:30:51-07:00" + }, + "created": "2000-08-21T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a2f13fd6-c438-41e6-bab8-6b1a4663f454" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2000-08-21T05:15:51-07:00", + "end": "2000-08-21T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b", + "resource": { + "resourceType": "Encounter", + "id": "e3bd9072-a533-48e6-ad52-8d79f0358f6b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2001-08-27T05:15:51-07:00", + "end": "2001-08-27T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2001-08-27T05:15:51-07:00", + "end": "2001-08-27T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f3d66454-5fa5-4f72-80da-eed86bfa1704", + "resource": { + "resourceType": "MedicationRequest", + "id": "f3d66454-5fa5-4f72-80da-eed86bfa1704", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" + }, + "authoredOn": "2001-08-27T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:65451aa1-cd45-4c73-a969-04e58d815bf9", + "resource": { + "resourceType": "Claim", + "id": "65451aa1-cd45-4c73-a969-04e58d815bf9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2001-08-27T05:15:51-07:00", + "end": "2001-08-27T05:30:51-07:00" + }, + "created": "2001-08-27T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:f3d66454-5fa5-4f72-80da-eed86bfa1704" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2626ec55-4590-4216-afa1-9b2e7848553a", + "resource": { + "resourceType": "Claim", + "id": "2626ec55-4590-4216-afa1-9b2e7848553a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2001-08-27T05:15:51-07:00", + "end": "2001-08-27T05:30:51-07:00" + }, + "created": "2001-08-27T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b6f36a29-c84a-467a-8395-934d9ab358d4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b6f36a29-c84a-467a-8395-934d9ab358d4", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2626ec55-4590-4216-afa1-9b2e7848553a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2001-08-27T05:30:51-07:00", + "end": "2002-08-27T05:30:51-07:00" + }, + "created": "2001-08-27T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2626ec55-4590-4216-afa1-9b2e7848553a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2001-08-27T05:15:51-07:00", + "end": "2001-08-27T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393", + "resource": { + "resourceType": "Encounter", + "id": "69d8df57-0ae8-442c-96ef-9bf824f0c393", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2002-09-02T05:15:51-07:00", + "end": "2002-09-02T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2002-09-02T05:15:51-07:00", + "end": "2002-09-02T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ccf20089-8271-4f70-9b81-4b8db495f873", + "resource": { + "resourceType": "MedicationRequest", + "id": "ccf20089-8271-4f70-9b81-4b8db495f873", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" + }, + "authoredOn": "2002-09-02T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:c63cf98c-45da-4708-940a-2ff8d0820ba9", + "resource": { + "resourceType": "Claim", + "id": "c63cf98c-45da-4708-940a-2ff8d0820ba9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2002-09-02T05:15:51-07:00", + "end": "2002-09-02T05:30:51-07:00" + }, + "created": "2002-09-02T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ccf20089-8271-4f70-9b81-4b8db495f873" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ed2a44a0-d34a-4d1a-90fb-98ee417f8686", + "resource": { + "resourceType": "Claim", + "id": "ed2a44a0-d34a-4d1a-90fb-98ee417f8686", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2002-09-02T05:15:51-07:00", + "end": "2002-09-02T05:30:51-07:00" + }, + "created": "2002-09-02T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:43482409-317b-4c09-b465-fbc2afba7d03", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "43482409-317b-4c09-b465-fbc2afba7d03", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ed2a44a0-d34a-4d1a-90fb-98ee417f8686" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2002-09-02T05:30:51-07:00", + "end": "2003-09-02T05:30:51-07:00" + }, + "created": "2002-09-02T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ed2a44a0-d34a-4d1a-90fb-98ee417f8686" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2002-09-02T05:15:51-07:00", + "end": "2002-09-02T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001", + "resource": { + "resourceType": "Encounter", + "id": "38734c49-0953-4f17-80fe-f16483bdf001", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2003-09-08T05:15:51-07:00", + "end": "2003-09-08T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2003-09-08T05:15:51-07:00", + "end": "2003-09-08T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7c36c44e-166b-4c3e-a473-b88793457aee", + "resource": { + "resourceType": "MedicationRequest", + "id": "7c36c44e-166b-4c3e-a473-b88793457aee", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" + }, + "authoredOn": "2003-09-08T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:4f090bfb-4409-433d-8546-c26faa0808f3", + "resource": { + "resourceType": "Claim", + "id": "4f090bfb-4409-433d-8546-c26faa0808f3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2003-09-08T05:15:51-07:00", + "end": "2003-09-08T05:30:51-07:00" + }, + "created": "2003-09-08T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:7c36c44e-166b-4c3e-a473-b88793457aee" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e5e73e04-4a53-4d57-8084-5152403ac862", + "resource": { + "resourceType": "Claim", + "id": "e5e73e04-4a53-4d57-8084-5152403ac862", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2003-09-08T05:15:51-07:00", + "end": "2003-09-08T05:30:51-07:00" + }, + "created": "2003-09-08T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5d39a1e4-f46b-4fa1-9e5b-f3ec8e827f43", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5d39a1e4-f46b-4fa1-9e5b-f3ec8e827f43", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e5e73e04-4a53-4d57-8084-5152403ac862" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2003-09-08T05:30:51-07:00", + "end": "2004-09-08T05:30:51-07:00" + }, + "created": "2003-09-08T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e5e73e04-4a53-4d57-8084-5152403ac862" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2003-09-08T05:15:51-07:00", + "end": "2003-09-08T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873", + "resource": { + "resourceType": "Encounter", + "id": "3c5c554f-f12a-4522-8ecd-9dd06f1c9873", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2004-09-13T05:15:51-07:00", + "end": "2004-09-13T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2004-09-13T05:15:51-07:00", + "end": "2004-09-13T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0e339852-87ed-4037-a76b-243b3ae4866d", + "resource": { + "resourceType": "MedicationRequest", + "id": "0e339852-87ed-4037-a76b-243b3ae4866d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" + }, + "authoredOn": "2004-09-13T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:fb07bf2a-0d66-46e1-be9a-4c652675e7e3", + "resource": { + "resourceType": "Claim", + "id": "fb07bf2a-0d66-46e1-be9a-4c652675e7e3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2004-09-13T05:15:51-07:00", + "end": "2004-09-13T05:45:51-07:00" + }, + "created": "2004-09-13T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:0e339852-87ed-4037-a76b-243b3ae4866d" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:37306189-78d5-43b4-a46a-e6b3fe59145e", + "resource": { + "resourceType": "Claim", + "id": "37306189-78d5-43b4-a46a-e6b3fe59145e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2004-09-13T05:15:51-07:00", + "end": "2004-09-13T05:45:51-07:00" + }, + "created": "2004-09-13T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:31684e94-d2dd-4534-ba52-308a2f150454", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "31684e94-d2dd-4534-ba52-308a2f150454", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "37306189-78d5-43b4-a46a-e6b3fe59145e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2004-09-13T05:45:51-07:00", + "end": "2005-09-13T05:45:51-07:00" + }, + "created": "2004-09-13T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:37306189-78d5-43b4-a46a-e6b3fe59145e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2004-09-13T05:15:51-07:00", + "end": "2004-09-13T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a", + "resource": { + "resourceType": "Encounter", + "id": "bde5f2d1-06f2-4ca1-8ca5-edf99867f02a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2005-09-19T05:15:51-07:00", + "end": "2005-09-19T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2005-09-19T05:15:51-07:00", + "end": "2005-09-19T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9ffc549f-07e9-4518-a20c-c39ea93316a9", + "resource": { + "resourceType": "MedicationRequest", + "id": "9ffc549f-07e9-4518-a20c-c39ea93316a9", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" + }, + "authoredOn": "2005-09-19T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:fcbd6b63-2bc3-4784-95c5-d0128f3b0509", + "resource": { + "resourceType": "Claim", + "id": "fcbd6b63-2bc3-4784-95c5-d0128f3b0509", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2005-09-19T05:15:51-07:00", + "end": "2005-09-19T05:45:51-07:00" + }, + "created": "2005-09-19T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:9ffc549f-07e9-4518-a20c-c39ea93316a9" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2f1294ad-5105-4447-aaa1-5277ee65be84", + "resource": { + "resourceType": "Claim", + "id": "2f1294ad-5105-4447-aaa1-5277ee65be84", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2005-09-19T05:15:51-07:00", + "end": "2005-09-19T05:45:51-07:00" + }, + "created": "2005-09-19T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:40dd5f2c-5f0d-4f7e-8cbd-af506b784ef0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "40dd5f2c-5f0d-4f7e-8cbd-af506b784ef0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2f1294ad-5105-4447-aaa1-5277ee65be84" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2005-09-19T05:45:51-07:00", + "end": "2006-09-19T05:45:51-07:00" + }, + "created": "2005-09-19T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2f1294ad-5105-4447-aaa1-5277ee65be84" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2005-09-19T05:15:51-07:00", + "end": "2005-09-19T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029", + "resource": { + "resourceType": "Encounter", + "id": "c7f5266a-1840-4fdc-bac1-39e12ab7d029", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2006-09-25T05:15:51-07:00", + "end": "2006-09-25T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2006-09-25T05:15:51-07:00", + "end": "2006-09-25T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c3e08e44-0734-4aa5-88d1-97b70e297e82", + "resource": { + "resourceType": "MedicationRequest", + "id": "c3e08e44-0734-4aa5-88d1-97b70e297e82", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" + }, + "authoredOn": "2006-09-25T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0e90715b-a68e-4cba-8097-7e02bde5ed0f", + "resource": { + "resourceType": "Claim", + "id": "0e90715b-a68e-4cba-8097-7e02bde5ed0f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2006-09-25T05:15:51-07:00", + "end": "2006-09-25T05:45:51-07:00" + }, + "created": "2006-09-25T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:c3e08e44-0734-4aa5-88d1-97b70e297e82" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:578cf805-ce3e-4b7e-879b-20fe1e733393", + "resource": { + "resourceType": "Claim", + "id": "578cf805-ce3e-4b7e-879b-20fe1e733393", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2006-09-25T05:15:51-07:00", + "end": "2006-09-25T05:45:51-07:00" + }, + "created": "2006-09-25T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7a45b794-7021-4b6a-bb27-becbbc7926c1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7a45b794-7021-4b6a-bb27-becbbc7926c1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "578cf805-ce3e-4b7e-879b-20fe1e733393" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2006-09-25T05:45:51-07:00", + "end": "2007-09-25T05:45:51-07:00" + }, + "created": "2006-09-25T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:578cf805-ce3e-4b7e-879b-20fe1e733393" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2006-09-25T05:15:51-07:00", + "end": "2006-09-25T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408", + "resource": { + "resourceType": "Encounter", + "id": "8da5a258-0811-482e-bf4f-ae41d8c9b408", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2007-10-01T05:15:51-07:00", + "end": "2007-10-01T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2007-10-01T05:15:51-07:00", + "end": "2007-10-01T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f42f87c9-776a-4e5b-8866-9b5a1c1ca873", + "resource": { + "resourceType": "MedicationRequest", + "id": "f42f87c9-776a-4e5b-8866-9b5a1c1ca873", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" + }, + "authoredOn": "2007-10-01T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5c6b001e-c236-4960-8753-d2ba6bd6810d", + "resource": { + "resourceType": "Claim", + "id": "5c6b001e-c236-4960-8753-d2ba6bd6810d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2007-10-01T05:15:51-07:00", + "end": "2007-10-01T05:45:51-07:00" + }, + "created": "2007-10-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:f42f87c9-776a-4e5b-8866-9b5a1c1ca873" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cc07e348-c328-45b7-b77d-5a84aee092a8", + "resource": { + "resourceType": "Claim", + "id": "cc07e348-c328-45b7-b77d-5a84aee092a8", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2007-10-01T05:15:51-07:00", + "end": "2007-10-01T05:45:51-07:00" + }, + "created": "2007-10-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7b74363c-446b-4a0a-9400-05e58af6f205", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7b74363c-446b-4a0a-9400-05e58af6f205", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cc07e348-c328-45b7-b77d-5a84aee092a8" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2007-10-01T05:45:51-07:00", + "end": "2008-10-01T05:45:51-07:00" + }, + "created": "2007-10-01T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:cc07e348-c328-45b7-b77d-5a84aee092a8" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2007-10-01T05:15:51-07:00", + "end": "2007-10-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66", + "resource": { + "resourceType": "Encounter", + "id": "d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2008-10-06T05:15:51-07:00", + "end": "2008-10-06T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2008-10-06T05:15:51-07:00", + "end": "2008-10-06T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:569719e7-88cc-4b94-b50d-0f1b09c3a241", + "resource": { + "resourceType": "MedicationRequest", + "id": "569719e7-88cc-4b94-b50d-0f1b09c3a241", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" + }, + "authoredOn": "2008-10-06T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:12ad2407-fc50-4bb1-bfb0-13f3c94a47c6", + "resource": { + "resourceType": "Claim", + "id": "12ad2407-fc50-4bb1-bfb0-13f3c94a47c6", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2008-10-06T05:15:51-07:00", + "end": "2008-10-06T05:30:51-07:00" + }, + "created": "2008-10-06T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:569719e7-88cc-4b94-b50d-0f1b09c3a241" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5e7c6df6-04f6-43ac-bcd3-3eda573322eb", + "resource": { + "resourceType": "Claim", + "id": "5e7c6df6-04f6-43ac-bcd3-3eda573322eb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2008-10-06T05:15:51-07:00", + "end": "2008-10-06T05:30:51-07:00" + }, + "created": "2008-10-06T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ca3c618e-d5bd-42f3-a387-bff54ef7ccef", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ca3c618e-d5bd-42f3-a387-bff54ef7ccef", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5e7c6df6-04f6-43ac-bcd3-3eda573322eb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2008-10-06T05:30:51-07:00", + "end": "2009-10-06T05:30:51-07:00" + }, + "created": "2008-10-06T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5e7c6df6-04f6-43ac-bcd3-3eda573322eb" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2008-10-06T05:15:51-07:00", + "end": "2008-10-06T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb", + "resource": { + "resourceType": "Encounter", + "id": "e64717ac-fe72-473a-a97d-ca260c8e79fb", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2009-10-12T05:15:51-07:00", + "end": "2009-10-12T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2009-10-12T05:15:51-07:00", + "end": "2009-10-12T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:735e3ecf-e9e6-4fc0-a560-73c7764aab82", + "resource": { + "resourceType": "MedicationRequest", + "id": "735e3ecf-e9e6-4fc0-a560-73c7764aab82", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" + }, + "authoredOn": "2009-10-12T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:21eb7a78-ab08-4f40-a6bb-5d6622661f42", + "resource": { + "resourceType": "Claim", + "id": "21eb7a78-ab08-4f40-a6bb-5d6622661f42", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2009-10-12T05:15:51-07:00", + "end": "2009-10-12T05:30:51-07:00" + }, + "created": "2009-10-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:735e3ecf-e9e6-4fc0-a560-73c7764aab82" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8b50a202-84e3-4d11-8d29-f722a213882b", + "resource": { + "resourceType": "Claim", + "id": "8b50a202-84e3-4d11-8d29-f722a213882b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2009-10-12T05:15:51-07:00", + "end": "2009-10-12T05:30:51-07:00" + }, + "created": "2009-10-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b85e16fc-b6f6-4a20-9489-0fd0eb1ee9d6", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b85e16fc-b6f6-4a20-9489-0fd0eb1ee9d6", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8b50a202-84e3-4d11-8d29-f722a213882b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2009-10-12T05:30:51-07:00", + "end": "2010-10-12T05:30:51-07:00" + }, + "created": "2009-10-12T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8b50a202-84e3-4d11-8d29-f722a213882b" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2009-10-12T05:15:51-07:00", + "end": "2009-10-12T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6", + "resource": { + "resourceType": "Encounter", + "id": "1222f480-de35-4581-8234-aac3e0aaebf6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:140de0d2-40c2-44ca-9f44-6f25bb639b21", + "resource": { + "resourceType": "Observation", + "id": "140de0d2-40c2-44ca-9f44-6f25bb639b21", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47c76101-3c84-4587-884e-1b15fa4ed4ef", + "resource": { + "resourceType": "Observation", + "id": "47c76101-3c84-4587-884e-1b15fa4ed4ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:edecffeb-3509-47ba-a1f1-b7ab7ee1e101", + "resource": { + "resourceType": "Observation", + "id": "edecffeb-3509-47ba-a1f1-b7ab7ee1e101", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4172e8e9-596f-4431-8238-15bbd2d545ae", + "resource": { + "resourceType": "Observation", + "id": "4172e8e9-596f-4431-8238-15bbd2d545ae", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f350d116-2f8e-484d-b5a2-ab30d6dc0153", + "resource": { + "resourceType": "Observation", + "id": "f350d116-2f8e-484d-b5a2-ab30d6dc0153", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 106, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36ea7764-6c28-492a-83ce-82d2ef9e882c", + "resource": { + "resourceType": "Observation", + "id": "36ea7764-6c28-492a-83ce-82d2ef9e882c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 79, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb9276b8-1438-4be4-8f91-5cab50a69f70", + "resource": { + "resourceType": "Observation", + "id": "cb9276b8-1438-4be4-8f91-5cab50a69f70", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e0ef354-df58-4853-b411-d4f5a8f89076", + "resource": { + "resourceType": "Observation", + "id": "4e0ef354-df58-4853-b411-d4f5a8f89076", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 70.01, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7866c71e-5535-4847-b40a-e612c3a7bb98", + "resource": { + "resourceType": "Observation", + "id": "7866c71e-5535-4847-b40a-e612c3a7bb98", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 14.38, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6c369d1-c352-489f-8c55-8d5bcb4c59c7", + "resource": { + "resourceType": "Observation", + "id": "b6c369d1-c352-489f-8c55-8d5bcb4c59c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.74, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fabfb47f-c5e3-42ec-9c96-0e3760a9214d", + "resource": { + "resourceType": "Observation", + "id": "fabfb47f-c5e3-42ec-9c96-0e3760a9214d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 8.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49303ecf-8654-40c8-a42f-1920ec4a730f", + "resource": { + "resourceType": "Observation", + "id": "49303ecf-8654-40c8-a42f-1920ec4a730f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 140.78, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c923c15-2f88-4160-9ddc-12c29dfb9ddb", + "resource": { + "resourceType": "Observation", + "id": "0c923c15-2f88-4160-9ddc-12c29dfb9ddb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.79, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb3fff56-00ab-449b-8bc0-37bac09fcaf0", + "resource": { + "resourceType": "Observation", + "id": "cb3fff56-00ab-449b-8bc0-37bac09fcaf0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 102.71, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50a33072-716e-487c-9937-c680d25c5611", + "resource": { + "resourceType": "Observation", + "id": "50a33072-716e-487c-9937-c680d25c5611", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 23.04, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b371c06-ad8e-4fff-95a7-1eda63211530", + "resource": { + "resourceType": "Observation", + "id": "0b371c06-ad8e-4fff-95a7-1eda63211530", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 169.45, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:243016dc-80a3-4051-b09a-29c4bcb8bd00", + "resource": { + "resourceType": "Observation", + "id": "243016dc-80a3-4051-b09a-29c4bcb8bd00", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 143.48, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", + "resource": { + "resourceType": "Observation", + "id": "4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 64.11, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30610131-6e4c-45ff-8f9e-e146cea910e4", + "resource": { + "resourceType": "Observation", + "id": "30610131-6e4c-45ff-8f9e-e146cea910e4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 76.64, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:55998799-c7ce-4284-b9e0-ce9401537c9a", + "resource": { + "resourceType": "Observation", + "id": "55998799-c7ce-4284-b9e0-ce9401537c9a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5839414d-a5c6-42d1-a580-11ff69a5089f", + "resource": { + "resourceType": "Observation", + "id": "5839414d-a5c6-42d1-a580-11ff69a5089f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.9, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b18c6bef-9d11-452f-999c-326f2bb47fc9", + "resource": { + "resourceType": "MedicationRequest", + "id": "b18c6bef-9d11-452f-999c-326f2bb47fc9", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "authoredOn": "2010-10-18T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:4fd918ac-d1f7-4dec-9cb4-9ac3d9a6bb66", + "resource": { + "resourceType": "Claim", + "id": "4fd918ac-d1f7-4dec-9cb4-9ac3d9a6bb66", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "created": "2010-10-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:b18c6bef-9d11-452f-999c-326f2bb47fc9" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:143f7dc2-52c7-4d88-b91b-f9f0a2f25871", + "resource": { + "resourceType": "Immunization", + "id": "143f7dc2-52c7-4d88-b91b-f9f0a2f25871", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "occurrenceDateTime": "2010-10-18T05:15:51-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d83664cb-16d7-4b34-b292-df1701735e1b", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d83664cb-16d7-4b34-b292-df1701735e1b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:4e0ef354-df58-4853-b411-d4f5a8f89076", + "display": "Glucose" + }, + { + "reference": "urn:uuid:7866c71e-5535-4847-b40a-e612c3a7bb98", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:b6c369d1-c352-489f-8c55-8d5bcb4c59c7", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:fabfb47f-c5e3-42ec-9c96-0e3760a9214d", + "display": "Calcium" + }, + { + "reference": "urn:uuid:49303ecf-8654-40c8-a42f-1920ec4a730f", + "display": "Sodium" + }, + { + "reference": "urn:uuid:0c923c15-2f88-4160-9ddc-12c29dfb9ddb", + "display": "Potassium" + }, + { + "reference": "urn:uuid:cb3fff56-00ab-449b-8bc0-37bac09fcaf0", + "display": "Chloride" + }, + { + "reference": "urn:uuid:50a33072-716e-487c-9937-c680d25c5611", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:d402ed47-c431-42b6-bbb5-909171ab56d5", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d402ed47-c431-42b6-bbb5-909171ab56d5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + }, + "effectiveDateTime": "2010-10-18T05:15:51-07:00", + "issued": "2010-10-18T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:0b371c06-ad8e-4fff-95a7-1eda63211530", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:243016dc-80a3-4051-b09a-29c4bcb8bd00", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:30610131-6e4c-45ff-8f9e-e146cea910e4", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:ea6972ba-efb0-40a7-9e89-79bb7431510f", + "resource": { + "resourceType": "Claim", + "id": "ea6972ba-efb0-40a7-9e89-79bb7431510f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "created": "2010-10-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:143f7dc2-52c7-4d88-b91b-f9f0a2f25871" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:66468a95-8439-476d-ad23-f87d92f5a129", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "66468a95-8439-476d-ad23-f87d92f5a129", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ea6972ba-efb0-40a7-9e89-79bb7431510f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2010-10-18T05:30:51-07:00", + "end": "2011-10-18T05:30:51-07:00" + }, + "created": "2010-10-18T05:30:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ea6972ba-efb0-40a7-9e89-79bb7431510f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2010-10-18T05:15:51-07:00", + "end": "2010-10-18T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560", + "resource": { + "resourceType": "Encounter", + "id": "e14a3dae-6bf7-4ecb-8c51-1ef2ff792560", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:da08dc9f-0ccd-4461-b501-3a0f5b0e3bce", + "resource": { + "resourceType": "Observation", + "id": "da08dc9f-0ccd-4461-b501-3a0f5b0e3bce", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff00b5aa-222d-4305-a633-13f637f87570", + "resource": { + "resourceType": "Observation", + "id": "ff00b5aa-222d-4305-a633-13f637f87570", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bcd47440-b08b-46fa-a255-d7890f350f94", + "resource": { + "resourceType": "Observation", + "id": "bcd47440-b08b-46fa-a255-d7890f350f94", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82475d55-eed8-4eca-ae6d-3da2a84b3a62", + "resource": { + "resourceType": "Observation", + "id": "82475d55-eed8-4eca-ae6d-3da2a84b3a62", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e561ae01-7419-4b58-b3a5-ea257b672791", + "resource": { + "resourceType": "Observation", + "id": "e561ae01-7419-4b58-b3a5-ea257b672791", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b92349e5-53ee-4b62-9f0d-a5096eee67b8", + "resource": { + "resourceType": "Observation", + "id": "b92349e5-53ee-4b62-9f0d-a5096eee67b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 84, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bace38ef-8a1d-4830-839c-70c64d7a6eaf", + "resource": { + "resourceType": "Observation", + "id": "bace38ef-8a1d-4830-839c-70c64d7a6eaf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", + "resource": { + "resourceType": "Observation", + "id": "c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 67.06, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7aab2a83-5c72-4bd5-bfae-e6cfff55725e", + "resource": { + "resourceType": "Observation", + "id": "7aab2a83-5c72-4bd5-bfae-e6cfff55725e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 8.68, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", + "resource": { + "resourceType": "Observation", + "id": "47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.75, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:62345264-3bbe-4cd9-8426-e66a2046ff45", + "resource": { + "resourceType": "Observation", + "id": "62345264-3bbe-4cd9-8426-e66a2046ff45", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.11, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", + "resource": { + "resourceType": "Observation", + "id": "ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 137.01, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9969b2b5-880a-4281-9f4f-1946b4028fa9", + "resource": { + "resourceType": "Observation", + "id": "9969b2b5-880a-4281-9f4f-1946b4028fa9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.09, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47f2462f-7edc-43c7-af3d-2664d3ccfc32", + "resource": { + "resourceType": "Observation", + "id": "47f2462f-7edc-43c7-af3d-2664d3ccfc32", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 106.45, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f73039a-d34c-4368-b9b6-a4e51fb69b97", + "resource": { + "resourceType": "Observation", + "id": "9f73039a-d34c-4368-b9b6-a4e51fb69b97", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 22.78, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1780db0a-b024-49b7-8f1d-4909f8a90f21", + "resource": { + "resourceType": "Observation", + "id": "1780db0a-b024-49b7-8f1d-4909f8a90f21", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:75b75ae3-8ac2-4f47-9d62-26b05f9f1600", + "resource": { + "resourceType": "Observation", + "id": "75b75ae3-8ac2-4f47-9d62-26b05f9f1600", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "valueQuantity": { + "value": 6.26, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:46bbd862-0fd4-4f10-abd5-16695c303df3", + "resource": { + "resourceType": "Procedure", + "id": "46bbd862-0fd4-4f10-abd5-16695c303df3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "performedPeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:30:51-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:dda7c2c4-a680-428b-bf27-4767b7b25ce0", + "resource": { + "resourceType": "MedicationRequest", + "id": "dda7c2c4-a680-428b-bf27-4767b7b25ce0", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "authoredOn": "2011-10-24T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:228d1384-5329-4bfd-87d3-5c1844b42517", + "resource": { + "resourceType": "Claim", + "id": "228d1384-5329-4bfd-87d3-5c1844b42517", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "created": "2011-10-24T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:dda7c2c4-a680-428b-bf27-4767b7b25ce0" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:191dbed8-5356-499f-8bcb-1f565f14b47d", + "resource": { + "resourceType": "Immunization", + "id": "191dbed8-5356-499f-8bcb-1f565f14b47d", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "occurrenceDateTime": "2011-10-24T05:15:51-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0f6c87ca-1a69-4ec2-b698-c37f5a4b0a46", + "resource": { + "resourceType": "DiagnosticReport", + "id": "0f6c87ca-1a69-4ec2-b698-c37f5a4b0a46", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + }, + "effectiveDateTime": "2011-10-24T05:15:51-07:00", + "issued": "2011-10-24T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", + "display": "Glucose" + }, + { + "reference": "urn:uuid:7aab2a83-5c72-4bd5-bfae-e6cfff55725e", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:62345264-3bbe-4cd9-8426-e66a2046ff45", + "display": "Calcium" + }, + { + "reference": "urn:uuid:ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", + "display": "Sodium" + }, + { + "reference": "urn:uuid:9969b2b5-880a-4281-9f4f-1946b4028fa9", + "display": "Potassium" + }, + { + "reference": "urn:uuid:47f2462f-7edc-43c7-af3d-2664d3ccfc32", + "display": "Chloride" + }, + { + "reference": "urn:uuid:9f73039a-d34c-4368-b9b6-a4e51fb69b97", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:5410ce3a-7f8f-40da-bc01-e4d5432967e2", + "resource": { + "resourceType": "Claim", + "id": "5410ce3a-7f8f-40da-bc01-e4d5432967e2", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "created": "2011-10-24T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:191dbed8-5356-499f-8bcb-1f565f14b47d" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:46bbd862-0fd4-4f10-abd5-16695c303df3" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 455.84, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:73f5da41-ea78-4cf7-9d9e-63187ddd30bb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "73f5da41-ea78-4cf7-9d9e-63187ddd30bb", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5410ce3a-7f8f-40da-bc01-e4d5432967e2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2011-10-24T05:45:51-07:00", + "end": "2012-10-24T05:45:51-07:00" + }, + "created": "2011-10-24T05:45:51-07:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5410ce3a-7f8f-40da-bc01-e4d5432967e2" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2011-10-24T05:15:51-07:00", + "end": "2011-10-24T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 455.84, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 91.168, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 364.672, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 455.84, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 455.84, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 477.088, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244", + "resource": { + "resourceType": "Encounter", + "id": "ccd45c67-85e0-48af-be78-95d9547f9244", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", + "resource": { + "resourceType": "Condition", + "id": "b6596ab7-d051-4237-a4d0-1d86c89e92b8", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68496003", + "display": "Polyp of colon" + } + ], + "text": "Polyp of colon" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "onsetDateTime": "2011-12-06T04:15:51-08:00", + "recordedDate": "2011-12-06T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b24b6a98-1bb8-4da9-87f4-9b099c008204", + "resource": { + "resourceType": "Observation", + "id": "b24b6a98-1bb8-4da9-87f4-9b099c008204", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33756-8", + "display": "Polyp size greatest dimension by CAP cancer protocols" + } + ], + "text": "Polyp size greatest dimension by CAP cancer protocols" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "effectiveDateTime": "2011-12-06T04:15:51-08:00", + "issued": "2011-12-06T04:15:51.927-08:00", + "valueQuantity": { + "value": 5.618, + "unit": "mm", + "system": "http://unitsofmeasure.org", + "code": "mm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6fa5382a-c18f-4e03-a616-3caa504a88ba", + "resource": { + "resourceType": "Observation", + "id": "6fa5382a-c18f-4e03-a616-3caa504a88ba", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57905-2", + "display": "Hemoglobin.gastrointestinal [Presence] in Stool by Immunologic method" + } + ], + "text": "Hemoglobin.gastrointestinal [Presence] in Stool by Immunologic method" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "effectiveDateTime": "2011-12-06T04:15:51-08:00", + "issued": "2011-12-06T04:15:51.927-08:00", + "valueQuantity": { + "value": 13.654, + "unit": "ng/mL", + "system": "http://unitsofmeasure.org", + "code": "ng/mL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ead9552-589f-41e0-a094-c344a46b355c", + "resource": { + "resourceType": "Procedure", + "id": "5ead9552-589f-41e0-a094-c344a46b355c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "performedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T04:42:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:61ad0c64-16f0-4b26-8011-97dc3aa65eb8", + "resource": { + "resourceType": "Procedure", + "id": "61ad0c64-16f0-4b26-8011-97dc3aa65eb8", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104435004", + "display": "Screening for occult blood in feces (procedure)" + } + ], + "text": "Screening for occult blood in feces (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "performedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T04:30:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", + "display": "Polyp of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:fef09971-e75e-40b6-a6db-5d722d89ff3d", + "resource": { + "resourceType": "Procedure", + "id": "fef09971-e75e-40b6-a6db-5d722d89ff3d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + }, + "performedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T04:30:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", + "display": "Polyp of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a6149938-4660-49a1-af81-ed03a6f62c3b", + "resource": { + "resourceType": "Claim", + "id": "a6149938-4660-49a1-af81-ed03a6f62c3b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "created": "2011-12-06T05:27:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:5ead9552-589f-41e0-a094-c344a46b355c" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:61ad0c64-16f0-4b26-8011-97dc3aa65eb8" + } + }, + { + "sequence": 3, + "procedureReference": { + "reference": "urn:uuid:fef09971-e75e-40b6-a6db-5d722d89ff3d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 8196.11, + "currency": "USD" + } + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68496003", + "display": "Polyp of colon" + } + ], + "text": "Polyp of colon" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104435004", + "display": "Screening for occult blood in feces (procedure)" + } + ], + "text": "Screening for occult blood in feces (procedure)" + }, + "net": { + "value": 8462.10, + "currency": "USD" + } + }, + { + "sequence": 5, + "procedureSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "net": { + "value": 11956.73, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0768d249-9d7f-4ab9-86aa-59401ad5f4a1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0768d249-9d7f-4ab9-86aa-59401ad5f4a1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "UnitedHealthcare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "UnitedHealthcare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a6149938-4660-49a1-af81-ed03a6f62c3b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2011-12-06T05:27:51-08:00", + "end": "2012-12-06T05:27:51-08:00" + }, + "created": "2011-12-06T05:27:51-08:00", + "insurer": { + "display": "UnitedHealthcare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a6149938-4660-49a1-af81-ed03a6f62c3b" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "UnitedHealthcare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 8196.11, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1639.2220000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 6556.888000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 8196.11, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 8196.11, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "68496003", + "display": "Polyp of colon" + } + ], + "text": "Polyp of colon" + }, + "servicedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "104435004", + "display": "Screening for occult blood in feces (procedure)" + } + ], + "text": "Screening for occult blood in feces (procedure)" + }, + "servicedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 8462.10, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1692.42, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 6769.68, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 8462.10, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 8462.10, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "servicedPeriod": { + "start": "2011-12-06T04:15:51-08:00", + "end": "2011-12-06T05:27:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 11956.73, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2391.346, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9565.384, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11956.73, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11956.73, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 22891.952, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f", + "resource": { + "resourceType": "Encounter", + "id": "3f5eebb6-0f33-4823-9858-0ab6d00bd21f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d4e39b43-4b12-4349-8094-bdd382316a5b", + "resource": { + "resourceType": "Observation", + "id": "d4e39b43-4b12-4349-8094-bdd382316a5b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d700845-89e0-4185-998b-fb008c222877", + "resource": { + "resourceType": "Observation", + "id": "6d700845-89e0-4185-998b-fb008c222877", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:257843c1-2d0e-44c2-873b-1e8d1344719a", + "resource": { + "resourceType": "Observation", + "id": "257843c1-2d0e-44c2-873b-1e8d1344719a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6890854a-49b2-4ba5-aa99-864b24664d0d", + "resource": { + "resourceType": "Observation", + "id": "6890854a-49b2-4ba5-aa99-864b24664d0d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:74976a7f-104a-472f-87be-03bbc3b93bd4", + "resource": { + "resourceType": "Observation", + "id": "74976a7f-104a-472f-87be-03bbc3b93bd4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e195dd89-9de4-4908-a001-866836151491", + "resource": { + "resourceType": "Observation", + "id": "e195dd89-9de4-4908-a001-866836151491", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 69, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ce435a3e-fa1d-47d5-9665-240da052d3ef", + "resource": { + "resourceType": "Observation", + "id": "ce435a3e-fa1d-47d5-9665-240da052d3ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6a40303d-051f-4fed-9607-d5ae00c08a4c", + "resource": { + "resourceType": "Observation", + "id": "6a40303d-051f-4fed-9607-d5ae00c08a4c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 83.7, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b27f29c-7faf-4197-a2ad-5350aaece63e", + "resource": { + "resourceType": "Observation", + "id": "6b27f29c-7faf-4197-a2ad-5350aaece63e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.98, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:44167af7-c65a-468c-8955-09709335d754", + "resource": { + "resourceType": "Observation", + "id": "44167af7-c65a-468c-8955-09709335d754", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.74, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", + "resource": { + "resourceType": "Observation", + "id": "6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 8.88, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:06c5782f-7435-4cf1-bb98-b0098630e4ea", + "resource": { + "resourceType": "Observation", + "id": "06c5782f-7435-4cf1-bb98-b0098630e4ea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 141.29, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:638739d4-0337-4127-af89-0ece8620a94f", + "resource": { + "resourceType": "Observation", + "id": "638739d4-0337-4127-af89-0ece8620a94f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.9, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", + "resource": { + "resourceType": "Observation", + "id": "c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 110.69, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a3920911-4744-44b8-973f-295ece827da7", + "resource": { + "resourceType": "Observation", + "id": "a3920911-4744-44b8-973f-295ece827da7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 20.17, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6fdd33ad-3a56-4711-bf57-801ea954c9a1", + "resource": { + "resourceType": "Observation", + "id": "6fdd33ad-3a56-4711-bf57-801ea954c9a1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65411e70-da5e-4137-866c-36291ce21747", + "resource": { + "resourceType": "Observation", + "id": "65411e70-da5e-4137-866c-36291ce21747", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.9, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed7806ea-1108-4cc4-8fb8-3ec4ffb30257", + "resource": { + "resourceType": "Procedure", + "id": "ed7806ea-1108-4cc4-8fb8-3ec4ffb30257", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "performedPeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:30:51-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:fdd51ea4-9921-4dc1-9acb-529172952993", + "resource": { + "resourceType": "MedicationRequest", + "id": "fdd51ea4-9921-4dc1-9acb-529172952993", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "authoredOn": "2012-10-29T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:b3a13b4b-06ed-4322-aa7d-7f261e764707", + "resource": { + "resourceType": "Claim", + "id": "b3a13b4b-06ed-4322-aa7d-7f261e764707", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "created": "2012-10-29T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:fdd51ea4-9921-4dc1-9acb-529172952993" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:370c4480-eebf-43df-9a05-fcba4c17d12b", + "resource": { + "resourceType": "Immunization", + "id": "370c4480-eebf-43df-9a05-fcba4c17d12b", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "occurrenceDateTime": "2012-10-29T05:15:51-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:61ce6171-9054-4aa8-ae88-572486a0fe48", + "resource": { + "resourceType": "Immunization", + "id": "61ce6171-9054-4aa8-ae88-572486a0fe48", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "occurrenceDateTime": "2012-10-29T05:15:51-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:cff7f7ea-206c-44d9-8e14-a79145327cb8", + "resource": { + "resourceType": "DiagnosticReport", + "id": "cff7f7ea-206c-44d9-8e14-a79145327cb8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + }, + "effectiveDateTime": "2012-10-29T05:15:51-07:00", + "issued": "2012-10-29T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:6a40303d-051f-4fed-9607-d5ae00c08a4c", + "display": "Glucose" + }, + { + "reference": "urn:uuid:6b27f29c-7faf-4197-a2ad-5350aaece63e", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:44167af7-c65a-468c-8955-09709335d754", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", + "display": "Calcium" + }, + { + "reference": "urn:uuid:06c5782f-7435-4cf1-bb98-b0098630e4ea", + "display": "Sodium" + }, + { + "reference": "urn:uuid:638739d4-0337-4127-af89-0ece8620a94f", + "display": "Potassium" + }, + { + "reference": "urn:uuid:c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", + "display": "Chloride" + }, + { + "reference": "urn:uuid:a3920911-4744-44b8-973f-295ece827da7", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4", + "resource": { + "resourceType": "Claim", + "id": "718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "created": "2012-10-29T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:370c4480-eebf-43df-9a05-fcba4c17d12b" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:61ce6171-9054-4aa8-ae88-572486a0fe48" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ed7806ea-1108-4cc4-8fb8-3ec4ffb30257" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 938.72, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5b5e55bf-ecb1-45cf-8836-f572c7d1d443", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5b5e55bf-ecb1-45cf-8836-f572c7d1d443", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2012-10-29T05:45:51-07:00", + "end": "2013-10-29T05:45:51-07:00" + }, + "created": "2012-10-29T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "servicedPeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2012-10-29T05:15:51-07:00", + "end": "2012-10-29T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 938.72, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 187.74400000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 750.9760000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 938.72, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 938.72, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 975.8080000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935", + "resource": { + "resourceType": "Encounter", + "id": "2ce6f12f-85b2-4092-8b24-d91d66313935", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758", + "resource": { + "resourceType": "Condition", + "id": "9e17df71-4a5a-4ce5-ae43-9ca30c031758", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "713197008", + "display": "Recurrent rectal polyp" + } + ], + "text": "Recurrent rectal polyp" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + }, + "onsetDateTime": "2013-01-29T04:15:51-08:00", + "recordedDate": "2013-01-29T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b02a3e53-1eda-4624-8607-f780cd226880", + "resource": { + "resourceType": "Observation", + "id": "b02a3e53-1eda-4624-8607-f780cd226880", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + }, + "effectiveDateTime": "2013-01-29T04:15:51-08:00", + "issued": "2013-01-29T04:15:51.927-08:00", + "valueQuantity": { + "value": 6, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ea277578-dab8-479d-8e25-bcf8eb7dc7a3", + "resource": { + "resourceType": "Procedure", + "id": "ea277578-dab8-479d-8e25-bcf8eb7dc7a3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + }, + "performedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T04:52:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e0a9b65b-019b-47fe-9bba-9b6cd3a14857", + "resource": { + "resourceType": "Procedure", + "id": "e0a9b65b-019b-47fe-9bba-9b6cd3a14857", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + }, + "performedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T04:30:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758", + "display": "Recurrent rectal polyp" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d7e5b0cd-099e-472d-bb18-722f3f77b6a4", + "resource": { + "resourceType": "Claim", + "id": "d7e5b0cd-099e-472d-bb18-722f3f77b6a4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "created": "2013-01-29T05:22:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ea277578-dab8-479d-8e25-bcf8eb7dc7a3" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:e0a9b65b-019b-47fe-9bba-9b6cd3a14857" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 23036.39, + "currency": "USD" + } + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "713197008", + "display": "Recurrent rectal polyp" + } + ], + "text": "Recurrent rectal polyp" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "net": { + "value": 12474.97, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6c835e0f-b9f1-4253-8aad-b01473276b07", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6c835e0f-b9f1-4253-8aad-b01473276b07", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d7e5b0cd-099e-472d-bb18-722f3f77b6a4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-01-29T05:22:51-08:00", + "end": "2014-01-29T05:22:51-08:00" + }, + "created": "2013-01-29T05:22:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:d7e5b0cd-099e-472d-bb18-722f3f77b6a4" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 23036.39, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 4607.278, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 18429.112, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 23036.39, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 23036.39, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "713197008", + "display": "Recurrent rectal polyp" + } + ], + "text": "Recurrent rectal polyp" + }, + "servicedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "274031008", + "display": "Rectal polypectomy" + } + ], + "text": "Rectal polypectomy" + }, + "servicedPeriod": { + "start": "2013-01-29T04:15:51-08:00", + "end": "2013-01-29T05:22:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 12474.97, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2494.994, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9979.976, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 12474.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 12474.97, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 28409.088000000003, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b", + "resource": { + "resourceType": "Encounter", + "id": "2fd76fad-d839-43b0-9bbf-2adc6a15983b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2013-02-01T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2013-02-01T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "resource": { + "resourceType": "Condition", + "id": "a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ], + "text": "Overlapping malignant neoplasm of colon" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" + }, + "onsetDateTime": "2013-02-01T04:15:51-08:00", + "abatementDateTime": "2015-07-23T05:15:51-07:00", + "recordedDate": "2013-02-01T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:2dd59121-7bfa-41f2-83e6-8c9ebabc74d9", + "resource": { + "resourceType": "CareTeam", + "id": "2dd59121-7bfa-41f2-83e6-8c9ebabc74d9", + "status": "inactive", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" + }, + "period": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2015-07-23T05:15:51-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ], + "text": "Overlapping malignant neoplasm of colon" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:49485686-7da2-4c6e-b039-14fd7c78f02c", + "resource": { + "resourceType": "CarePlan", + "id": "49485686-7da2-4c6e-b039-14fd7c78f02c", + "text": { + "status": "generated", + "div": "
    Care Plan for Cancer care plan.
    Activities:
    • Cancer care plan
    • Cancer care plan
    • Cancer care plan

    Care plan is meant to treat Overlapping malignant neoplasm of colon.
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "395082007", + "display": "Cancer care plan" + } + ], + "text": "Cancer care plan" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" + }, + "period": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2015-07-23T05:15:51-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:2dd59121-7bfa-41f2-83e6-8c9ebabc74d9" + } + ], + "addresses": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "226234005", + "display": "Healthy diet" + } + ], + "text": "Healthy diet" + }, + "status": "completed", + "location": { + "display": "CAPE COD HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703993001", + "display": "Colonoscopy planned" + } + ], + "text": "Colonoscopy planned" + }, + "status": "completed", + "location": { + "display": "CAPE COD HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "243072006", + "display": "Cancer education" + } + ], + "text": "Cancer education" + }, + "status": "completed", + "location": { + "display": "CAPE COD HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:792060bc-036f-4266-a268-cf45426684cb", + "resource": { + "resourceType": "Claim", + "id": "792060bc-036f-4266-a268-cf45426684cb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2013-02-01T04:30:51-08:00" + }, + "created": "2013-02-01T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ], + "text": "Overlapping malignant neoplasm of colon" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:11508325-4f1d-4d58-b692-579080484764", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "11508325-4f1d-4d58-b692-579080484764", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "792060bc-036f-4266-a268-cf45426684cb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-02-01T04:30:51-08:00", + "end": "2014-02-01T04:30:51-08:00" + }, + "created": "2013-02-01T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:792060bc-036f-4266-a268-cf45426684cb" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2013-02-01T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ], + "text": "Overlapping malignant neoplasm of colon" + }, + "servicedPeriod": { + "start": "2013-02-01T04:15:51-08:00", + "end": "2013-02-01T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403", + "resource": { + "resourceType": "Encounter", + "id": "4809ff01-1e8c-42d2-a7f1-32f786aaa403", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-09T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-09T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cc9e49a5-5011-4785-9674-e75cd8f6834c", + "resource": { + "resourceType": "Observation", + "id": "cc9e49a5-5011-4785-9674-e75cd8f6834c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + }, + "effectiveDateTime": "2013-02-08T04:15:51-08:00", + "issued": "2013-02-08T04:15:51.927-08:00", + "valueQuantity": { + "value": 6, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:313e470a-a6a1-4e8c-9feb-0ff493f359bb", + "resource": { + "resourceType": "Procedure", + "id": "313e470a-a6a1-4e8c-9feb-0ff493f359bb", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43075005", + "display": "Partial resection of colon" + } + ], + "text": "Partial resection of colon" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + }, + "performedPeriod": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-08T04:30:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a4792bae-b76e-48b9-9f34-4de61a60bdbb", + "resource": { + "resourceType": "CareTeam", + "id": "a4792bae-b76e-48b9-9f34-4de61a60bdbb", + "status": "inactive", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + }, + "period": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-19T04:15:51-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ], + "text": "Overlapping malignant neoplasm of colon" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:a9820028-405f-459c-b510-fae895c22f47", + "resource": { + "resourceType": "CarePlan", + "id": "a9820028-405f-459c-b510-fae895c22f47", + "text": { + "status": "generated", + "div": "
    Care Plan for Major surgery care management.
    Activities:
    • Major surgery care management
    • Major surgery care management

    Care plan is meant to treat Overlapping malignant neoplasm of colon.
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "781831000000109", + "display": "Major surgery care management" + } + ], + "text": "Major surgery care management" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + }, + "period": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-19T04:15:51-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:a4792bae-b76e-48b9-9f34-4de61a60bdbb" + } + ], + "addresses": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10888001", + "display": "Liquid diet" + } + ], + "text": "Liquid diet" + }, + "status": "completed", + "location": { + "display": "CAPE COD HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "103744005", + "display": "Administration of intravenous fluids" + } + ], + "text": "Administration of intravenous fluids" + }, + "status": "completed", + "location": { + "display": "CAPE COD HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:010e4ade-c95c-4b50-a84b-cf66f3893ddf", + "resource": { + "resourceType": "Claim", + "id": "010e4ade-c95c-4b50-a84b-cf66f3893ddf", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-09T04:30:51-08:00" + }, + "created": "2013-02-09T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:313e470a-a6a1-4e8c-9feb-0ff493f359bb" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43075005", + "display": "Partial resection of colon" + } + ], + "text": "Partial resection of colon" + }, + "net": { + "value": 38424.92, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:db5b770b-726c-40fd-bac2-795664a67023", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "db5b770b-726c-40fd-bac2-795664a67023", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "010e4ade-c95c-4b50-a84b-cf66f3893ddf" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-02-09T04:30:51-08:00", + "end": "2014-02-09T04:30:51-08:00" + }, + "created": "2013-02-09T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:010e4ade-c95c-4b50-a84b-cf66f3893ddf" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-09T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43075005", + "display": "Partial resection of colon" + } + ], + "text": "Partial resection of colon" + }, + "servicedPeriod": { + "start": "2013-02-08T04:15:51-08:00", + "end": "2013-02-09T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 38424.92, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 7684.984, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 30739.936, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 38424.92, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 38424.92, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 30739.936, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476", + "resource": { + "resourceType": "Encounter", + "id": "2d926120-545e-45bd-bb18-1bd83b3ec476", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a5f4315a-3bb6-416e-b44f-c7a6f938ac6c", + "resource": { + "resourceType": "Observation", + "id": "a5f4315a-3bb6-416e-b44f-c7a6f938ac6c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 6, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6d01daa-1018-4dbc-9b2c-8a0a24029d25", + "resource": { + "resourceType": "Observation", + "id": "b6d01daa-1018-4dbc-9b2c-8a0a24029d25", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 89.55, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:55124c0f-5b70-45ea-b4a9-30cc4eb883ba", + "resource": { + "resourceType": "Observation", + "id": "55124c0f-5b70-45ea-b4a9-30cc4eb883ba", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 10.77, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:02a11d04-ad54-4bfa-9bd6-f57760863136", + "resource": { + "resourceType": "Observation", + "id": "02a11d04-ad54-4bfa-9bd6-f57760863136", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 2.5003, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebd2b5d8-5411-43de-bf84-d065d60e6986", + "resource": { + "resourceType": "Observation", + "id": "ebd2b5d8-5411-43de-bf84-d065d60e6986", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.5, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53a0e1bc-650e-4dfb-9ba4-74c211160b03", + "resource": { + "resourceType": "Observation", + "id": "53a0e1bc-650e-4dfb-9ba4-74c211160b03", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 138.42, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36c36666-4a04-415c-9951-dbbcd5e3c631", + "resource": { + "resourceType": "Observation", + "id": "36c36666-4a04-415c-9951-dbbcd5e3c631", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.52, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:38de03a9-5ee7-41fb-86ca-1a95b507bbd7", + "resource": { + "resourceType": "Observation", + "id": "38de03a9-5ee7-41fb-86ca-1a95b507bbd7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 102.93, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", + "resource": { + "resourceType": "Observation", + "id": "4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 23.88, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77456a94-0af1-4cdd-86e7-e3748082cecc", + "resource": { + "resourceType": "Observation", + "id": "77456a94-0af1-4cdd-86e7-e3748082cecc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 17.566, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db36842c-234e-4f37-bba9-9e60b5ed6c7d", + "resource": { + "resourceType": "Observation", + "id": "db36842c-234e-4f37-bba9-9e60b5ed6c7d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 64.571, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d57e0fc-8f13-49b0-a9e8-76de6c37850c", + "resource": { + "resourceType": "Observation", + "id": "2d57e0fc-8f13-49b0-a9e8-76de6c37850c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 5.2204, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4ad163fb-192a-4af3-b4bf-94f439510b9b", + "resource": { + "resourceType": "Observation", + "id": "4ad163fb-192a-4af3-b4bf-94f439510b9b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 3.1771, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2e07e21f-5869-4904-ac49-6de060667641", + "resource": { + "resourceType": "Observation", + "id": "2e07e21f-5869-4904-ac49-6de060667641", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.18474, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1aa9cbcb-216c-4e4e-ac58-95432082003f", + "resource": { + "resourceType": "Observation", + "id": "1aa9cbcb-216c-4e4e-ac58-95432082003f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 133.07, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e3bcd75-77c9-4302-b1c6-556595550cd1", + "resource": { + "resourceType": "Observation", + "id": "3e3bcd75-77c9-4302-b1c6-556595550cd1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 51.686, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", + "resource": { + "resourceType": "Observation", + "id": "bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 12.513, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8c6a2800-a7d8-4471-906b-22a3fb51b313", + "resource": { + "resourceType": "Observation", + "id": "8c6a2800-a7d8-4471-906b-22a3fb51b313", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 2.6884, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ded636c1-e6a1-4f0f-8fe2-85a95748236a", + "resource": { + "resourceType": "Observation", + "id": "ded636c1-e6a1-4f0f-8fe2-85a95748236a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.6605, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4f8dee98-f03f-4fc5-b7dd-d9841c921164", + "resource": { + "resourceType": "Observation", + "id": "4f8dee98-f03f-4fc5-b7dd-d9841c921164", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 12.236, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d06a97e8-696f-4a05-a5e1-b0a962296fab", + "resource": { + "resourceType": "Observation", + "id": "d06a97e8-696f-4a05-a5e1-b0a962296fab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.069, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f535920-cd6d-4842-a1eb-2dbd73214b17", + "resource": { + "resourceType": "Observation", + "id": "9f535920-cd6d-4842-a1eb-2dbd73214b17", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 90.317, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c6a078c-2a08-47b2-93f3-689ff7e2987f", + "resource": { + "resourceType": "Observation", + "id": "9c6a078c-2a08-47b2-93f3-689ff7e2987f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 32.23, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", + "resource": { + "resourceType": "Observation", + "id": "f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 35.221, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d3e1e40b-5743-4ed5-b047-3f548131ded9", + "resource": { + "resourceType": "Observation", + "id": "d3e1e40b-5743-4ed5-b047-3f548131ded9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 41.158, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", + "resource": { + "resourceType": "Observation", + "id": "5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 438.01, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:08e428c1-6d81-4ea1-bd75-d33265bbd37d", + "resource": { + "resourceType": "Observation", + "id": "08e428c1-6d81-4ea1-bd75-d33265bbd37d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 273.43, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecf51aa3-5928-418d-9411-72618c5298fb", + "resource": { + "resourceType": "Observation", + "id": "ecf51aa3-5928-418d-9411-72618c5298fb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "valueQuantity": { + "value": 10.09, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25f6ccb3-6e71-41a1-a3ff-97ad95940745", + "resource": { + "resourceType": "Procedure", + "id": "25f6ccb3-6e71-41a1-a3ff-97ad95940745", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "performedPeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:30:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a3d098ca-f27c-497d-b484-e78f34e7652b", + "resource": { + "resourceType": "MedicationRequest", + "id": "a3d098ca-f27c-497d-b484-e78f34e7652b", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1803932", + "display": "Leucovorin 100 MG Injection" + } + ], + "text": "Leucovorin 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "authoredOn": "2013-02-26T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5a30cba6-ec71-4a88-a6c5-6a2c4429218e", + "resource": { + "resourceType": "Claim", + "id": "5a30cba6-ec71-4a88-a6c5-6a2c4429218e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "created": "2013-02-26T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:a3d098ca-f27c-497d-b484-e78f34e7652b" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + } + ] + } + ], + "total": { + "value": 63.93, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:22394b83-909f-4c4c-8e72-195faf344c64", + "resource": { + "resourceType": "MedicationRequest", + "id": "22394b83-909f-4c4c-8e72-195faf344c64", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1736776", + "display": "10 ML oxaliplatin 5 MG/ML Injection" + } + ], + "text": "10 ML oxaliplatin 5 MG/ML Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "authoredOn": "2013-02-26T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:26da09d2-af4b-4ee4-945a-a3dda5fc9243", + "resource": { + "resourceType": "Claim", + "id": "26da09d2-af4b-4ee4-945a-a3dda5fc9243", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "created": "2013-02-26T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:22394b83-909f-4c4c-8e72-195faf344c64" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + } + ] + } + ], + "total": { + "value": 2507.24, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b7662618-21d8-4e39-b571-3a853e1b0d86", + "resource": { + "resourceType": "DiagnosticReport", + "id": "b7662618-21d8-4e39-b571-3a853e1b0d86", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:b6d01daa-1018-4dbc-9b2c-8a0a24029d25", + "display": "Glucose" + }, + { + "reference": "urn:uuid:55124c0f-5b70-45ea-b4a9-30cc4eb883ba", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:02a11d04-ad54-4bfa-9bd6-f57760863136", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:ebd2b5d8-5411-43de-bf84-d065d60e6986", + "display": "Calcium" + }, + { + "reference": "urn:uuid:53a0e1bc-650e-4dfb-9ba4-74c211160b03", + "display": "Sodium" + }, + { + "reference": "urn:uuid:36c36666-4a04-415c-9951-dbbcd5e3c631", + "display": "Potassium" + }, + { + "reference": "urn:uuid:38de03a9-5ee7-41fb-86ca-1a95b507bbd7", + "display": "Chloride" + }, + { + "reference": "urn:uuid:4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:77456a94-0af1-4cdd-86e7-e3748082cecc", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:db36842c-234e-4f37-bba9-9e60b5ed6c7d", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:2d57e0fc-8f13-49b0-a9e8-76de6c37850c", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4ad163fb-192a-4af3-b4bf-94f439510b9b", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:2e07e21f-5869-4904-ac49-6de060667641", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:1aa9cbcb-216c-4e4e-ac58-95432082003f", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:3e3bcd75-77c9-4302-b1c6-556595550cd1", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7dacebf4-2456-454a-b1ec-a804c4e69da2", + "resource": { + "resourceType": "DiagnosticReport", + "id": "7dacebf4-2456-454a-b1ec-a804c4e69da2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + }, + "effectiveDateTime": "2013-02-26T04:15:51-08:00", + "issued": "2013-02-26T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:8c6a2800-a7d8-4471-906b-22a3fb51b313", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:ded636c1-e6a1-4f0f-8fe2-85a95748236a", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:4f8dee98-f03f-4fc5-b7dd-d9841c921164", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:d06a97e8-696f-4a05-a5e1-b0a962296fab", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:9f535920-cd6d-4842-a1eb-2dbd73214b17", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:9c6a078c-2a08-47b2-93f3-689ff7e2987f", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:d3e1e40b-5743-4ed5-b047-3f548131ded9", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:08e428c1-6d81-4ea1-bd75-d33265bbd37d", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:ecf51aa3-5928-418d-9411-72618c5298fb", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0752ea3e-d40e-40d1-b5c7-fc33171125ca", + "resource": { + "resourceType": "Claim", + "id": "0752ea3e-d40e-40d1-b5c7-fc33171125ca", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "created": "2013-02-26T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:25f6ccb3-6e71-41a1-a3ff-97ad95940745" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 7161.00, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8beec78a-24b2-4b47-abda-b837cfbb259f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8beec78a-24b2-4b47-abda-b837cfbb259f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0752ea3e-d40e-40d1-b5c7-fc33171125ca" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-02-26T04:45:51-08:00", + "end": "2014-02-26T04:45:51-08:00" + }, + "created": "2013-02-26T04:45:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0752ea3e-d40e-40d1-b5c7-fc33171125ca" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-02-26T04:15:51-08:00", + "end": "2013-02-26T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 7161.00, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1432.2, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 5728.8, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 7161.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 7161.00, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 5728.8, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75", + "resource": { + "resourceType": "Encounter", + "id": "767da137-22c0-4811-8d09-af08dfda2f75", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:45:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:acaeefb1-9a5b-41b6-8f3a-5e5d3448fd5f", + "resource": { + "resourceType": "Observation", + "id": "acaeefb1-9a5b-41b6-8f3a-5e5d3448fd5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d37dc6f-e655-4d03-a64a-bfb7b32c4947", + "resource": { + "resourceType": "Observation", + "id": "7d37dc6f-e655-4d03-a64a-bfb7b32c4947", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 96.75, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2372beda-927e-460f-9de0-561b20d4c7a9", + "resource": { + "resourceType": "Observation", + "id": "2372beda-927e-460f-9de0-561b20d4c7a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 16.01, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c5cf511-b90d-4375-a10d-4b2c38c5e448", + "resource": { + "resourceType": "Observation", + "id": "1c5cf511-b90d-4375-a10d-4b2c38c5e448", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.8776, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8476f78-77fc-41dc-a1e1-482c0eb5a825", + "resource": { + "resourceType": "Observation", + "id": "b8476f78-77fc-41dc-a1e1-482c0eb5a825", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.48, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:75d3c18e-f2c0-4f63-b7d5-0a1000833542", + "resource": { + "resourceType": "Observation", + "id": "75d3c18e-f2c0-4f63-b7d5-0a1000833542", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 139.08, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71f65388-c251-40b6-bed7-72ce23d07ab5", + "resource": { + "resourceType": "Observation", + "id": "71f65388-c251-40b6-bed7-72ce23d07ab5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.8, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f9f36cee-0e35-4c5d-b8e1-f54c66269135", + "resource": { + "resourceType": "Observation", + "id": "f9f36cee-0e35-4c5d-b8e1-f54c66269135", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 106.95, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0161429-03f9-44c1-9271-2b5f55c7b4bb", + "resource": { + "resourceType": "Observation", + "id": "c0161429-03f9-44c1-9271-2b5f55c7b4bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 22.13, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25fd4b9c-4884-4704-b4c9-f7451c8e0869", + "resource": { + "resourceType": "Observation", + "id": "25fd4b9c-4884-4704-b4c9-f7451c8e0869", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 6.4972, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d981902d-784f-4484-8076-1321f1f5728e", + "resource": { + "resourceType": "Observation", + "id": "d981902d-784f-4484-8076-1321f1f5728e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 76.426, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4cf77f06-37f8-4a18-8424-08ff79bc6295", + "resource": { + "resourceType": "Observation", + "id": "4cf77f06-37f8-4a18-8424-08ff79bc6295", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.0969, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:235269bd-32c8-493e-817d-be726e4d5bcc", + "resource": { + "resourceType": "Observation", + "id": "235269bd-32c8-493e-817d-be726e4d5bcc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.1057, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9312514-fdc3-463e-ba7b-3d8ad8505a51", + "resource": { + "resourceType": "Observation", + "id": "c9312514-fdc3-463e-ba7b-3d8ad8505a51", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 1.0496, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b256ec36-7b93-45fa-b918-903299c12c73", + "resource": { + "resourceType": "Observation", + "id": "b256ec36-7b93-45fa-b918-903299c12c73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 75.286, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6153779-6dc8-4b3c-b932-b60afc3ee932", + "resource": { + "resourceType": "Observation", + "id": "b6153779-6dc8-4b3c-b932-b60afc3ee932", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 40.053, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4134cffe-0cc9-4398-aa37-75f505447298", + "resource": { + "resourceType": "Observation", + "id": "4134cffe-0cc9-4398-aa37-75f505447298", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.075, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4eb068e8-009d-418f-adb0-48cce5e82f21", + "resource": { + "resourceType": "Observation", + "id": "4eb068e8-009d-418f-adb0-48cce5e82f21", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.6354, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", + "resource": { + "resourceType": "Observation", + "id": "a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.9082, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c301ef52-3052-459b-89b4-6970154fa4f5", + "resource": { + "resourceType": "Observation", + "id": "c301ef52-3052-459b-89b4-6970154fa4f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.821, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7f38e37a-1be9-4e8a-8298-bca8fee88d50", + "resource": { + "resourceType": "Observation", + "id": "7f38e37a-1be9-4e8a-8298-bca8fee88d50", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 25.788, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a013b221-e582-4ab0-aeb0-78864817cdb2", + "resource": { + "resourceType": "Observation", + "id": "a013b221-e582-4ab0-aeb0-78864817cdb2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 89.699, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0438335c-3f9f-44a3-a4d9-2303a515dbab", + "resource": { + "resourceType": "Observation", + "id": "0438335c-3f9f-44a3-a4d9-2303a515dbab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 31.577, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:038d3409-dc60-4d71-8ec7-f5b6b3940997", + "resource": { + "resourceType": "Observation", + "id": "038d3409-dc60-4d71-8ec7-f5b6b3940997", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 35.19, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c78cb64-c435-4389-8cc4-2d824dad05d3", + "resource": { + "resourceType": "Observation", + "id": "5c78cb64-c435-4389-8cc4-2d824dad05d3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 40.449, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8dbebf8a-c9a9-490b-a5e6-260fb88bd456", + "resource": { + "resourceType": "Observation", + "id": "8dbebf8a-c9a9-490b-a5e6-260fb88bd456", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 249.37, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2e0739b-b9c1-450e-b3cf-d3ab91666e03", + "resource": { + "resourceType": "Observation", + "id": "b2e0739b-b9c1-450e-b3cf-d3ab91666e03", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 175.47, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d75282c-5c83-4c3c-8d57-d803af7cdb70", + "resource": { + "resourceType": "Observation", + "id": "7d75282c-5c83-4c3c-8d57-d803af7cdb70", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 11.843, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3359d124-1545-4f4e-ace9-7160b5137a6f", + "resource": { + "resourceType": "Procedure", + "id": "3359d124-1545-4f4e-ace9-7160b5137a6f", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "performedPeriod": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:30:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:87b9b981-ff42-4eca-8c77-02b964ac4739", + "resource": { + "resourceType": "DiagnosticReport", + "id": "87b9b981-ff42-4eca-8c77-02b964ac4739", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:7d37dc6f-e655-4d03-a64a-bfb7b32c4947", + "display": "Glucose" + }, + { + "reference": "urn:uuid:2372beda-927e-460f-9de0-561b20d4c7a9", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:1c5cf511-b90d-4375-a10d-4b2c38c5e448", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:b8476f78-77fc-41dc-a1e1-482c0eb5a825", + "display": "Calcium" + }, + { + "reference": "urn:uuid:75d3c18e-f2c0-4f63-b7d5-0a1000833542", + "display": "Sodium" + }, + { + "reference": "urn:uuid:71f65388-c251-40b6-bed7-72ce23d07ab5", + "display": "Potassium" + }, + { + "reference": "urn:uuid:f9f36cee-0e35-4c5d-b8e1-f54c66269135", + "display": "Chloride" + }, + { + "reference": "urn:uuid:c0161429-03f9-44c1-9271-2b5f55c7b4bb", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:25fd4b9c-4884-4704-b4c9-f7451c8e0869", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:d981902d-784f-4484-8076-1321f1f5728e", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4cf77f06-37f8-4a18-8424-08ff79bc6295", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:235269bd-32c8-493e-817d-be726e4d5bcc", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:c9312514-fdc3-463e-ba7b-3d8ad8505a51", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:b256ec36-7b93-45fa-b918-903299c12c73", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:b6153779-6dc8-4b3c-b932-b60afc3ee932", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4134cffe-0cc9-4398-aa37-75f505447298", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:30a3cbc5-dec0-4e3d-92d1-93e71f5d7539", + "resource": { + "resourceType": "DiagnosticReport", + "id": "30a3cbc5-dec0-4e3d-92d1-93e71f5d7539", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + }, + "effectiveDateTime": "2013-04-01T05:15:51-07:00", + "issued": "2013-04-01T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:4eb068e8-009d-418f-adb0-48cce5e82f21", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:c301ef52-3052-459b-89b4-6970154fa4f5", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:7f38e37a-1be9-4e8a-8298-bca8fee88d50", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:a013b221-e582-4ab0-aeb0-78864817cdb2", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:0438335c-3f9f-44a3-a4d9-2303a515dbab", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:038d3409-dc60-4d71-8ec7-f5b6b3940997", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:5c78cb64-c435-4389-8cc4-2d824dad05d3", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:8dbebf8a-c9a9-490b-a5e6-260fb88bd456", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b2e0739b-b9c1-450e-b3cf-d3ab91666e03", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:7d75282c-5c83-4c3c-8d57-d803af7cdb70", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:5e7abff6-7cd8-4e11-ba75-995e392e8435", + "resource": { + "resourceType": "Claim", + "id": "5e7abff6-7cd8-4e11-ba75-995e392e8435", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:45:51-07:00" + }, + "created": "2013-04-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:3359d124-1545-4f4e-ace9-7160b5137a6f" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 11924.17, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b4f3bdf3-df2f-47f2-ace1-8e47e07ef2b3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b4f3bdf3-df2f-47f2-ace1-8e47e07ef2b3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5e7abff6-7cd8-4e11-ba75-995e392e8435" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-04-01T05:45:51-07:00", + "end": "2014-04-01T05:45:51-07:00" + }, + "created": "2013-04-01T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5e7abff6-7cd8-4e11-ba75-995e392e8435" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-04-01T05:15:51-07:00", + "end": "2013-04-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 11924.17, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2384.8340000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9539.336000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11924.17, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11924.17, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 9539.336000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977", + "resource": { + "resourceType": "Encounter", + "id": "7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:45:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3f56b948-7137-4362-96dc-4d481403696d", + "resource": { + "resourceType": "Observation", + "id": "3f56b948-7137-4362-96dc-4d481403696d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 5, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db483d27-9d6b-4a8a-8eae-822d4bc480fa", + "resource": { + "resourceType": "Observation", + "id": "db483d27-9d6b-4a8a-8eae-822d4bc480fa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 91.62, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bdb81812-1403-4cc5-b70f-858f8a50984a", + "resource": { + "resourceType": "Observation", + "id": "bdb81812-1403-4cc5-b70f-858f8a50984a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.28, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:435c85d0-da18-4a27-b2c1-3c43acbffe38", + "resource": { + "resourceType": "Observation", + "id": "435c85d0-da18-4a27-b2c1-3c43acbffe38", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.2633, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c05fead5-b200-4676-8dbb-534a9a88837a", + "resource": { + "resourceType": "Observation", + "id": "c05fead5-b200-4676-8dbb-534a9a88837a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.05, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b28bd42-0541-4b8a-827b-01f90e142e15", + "resource": { + "resourceType": "Observation", + "id": "3b28bd42-0541-4b8a-827b-01f90e142e15", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 138.5, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e0230357-77ff-4a2a-a241-9e6f79d8d8e6", + "resource": { + "resourceType": "Observation", + "id": "e0230357-77ff-4a2a-a241-9e6f79d8d8e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.03, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", + "resource": { + "resourceType": "Observation", + "id": "4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 103.49, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c392cac0-c1cb-4676-8620-bddfa9e59dff", + "resource": { + "resourceType": "Observation", + "id": "c392cac0-c1cb-4676-8620-bddfa9e59dff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 25.95, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:890c6989-1e4a-4fe4-9ab5-cb1095d5437d", + "resource": { + "resourceType": "Observation", + "id": "890c6989-1e4a-4fe4-9ab5-cb1095d5437d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 14.388, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7562b88d-3f43-4089-95af-bf82c68f31d8", + "resource": { + "resourceType": "Observation", + "id": "7562b88d-3f43-4089-95af-bf82c68f31d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 69.584, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c440773-3dff-4822-a315-46fc30f099c6", + "resource": { + "resourceType": "Observation", + "id": "4c440773-3dff-4822-a315-46fc30f099c6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.4058, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", + "resource": { + "resourceType": "Observation", + "id": "4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.4536, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cdde0e90-236d-4d49-b6b6-64c010a9b118", + "resource": { + "resourceType": "Observation", + "id": "cdde0e90-236d-4d49-b6b6-64c010a9b118", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.76234, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0213fc21-7fde-4d6b-9043-0231c3f8eac5", + "resource": { + "resourceType": "Observation", + "id": "0213fc21-7fde-4d6b-9043-0231c3f8eac5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 50.534, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f0691777-00ab-49ce-b56d-ada1036eeab5", + "resource": { + "resourceType": "Observation", + "id": "f0691777-00ab-49ce-b56d-ada1036eeab5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 57.686, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:199a0c22-29cf-4e5d-9d05-3ba361fd8687", + "resource": { + "resourceType": "Observation", + "id": "199a0c22-29cf-4e5d-9d05-3ba361fd8687", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 32.071, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5654fec6-cf82-4e31-afcf-a53507f24661", + "resource": { + "resourceType": "Observation", + "id": "5654fec6-cf82-4e31-afcf-a53507f24661", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.0567, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", + "resource": { + "resourceType": "Observation", + "id": "12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.9416, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb6f8e83-79cf-4508-ba06-a156a966eb8a", + "resource": { + "resourceType": "Observation", + "id": "fb6f8e83-79cf-4508-ba06-a156a966eb8a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 12.142, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba58aa20-515d-41e7-a436-c48af6109029", + "resource": { + "resourceType": "Observation", + "id": "ba58aa20-515d-41e7-a436-c48af6109029", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 25.632, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", + "resource": { + "resourceType": "Observation", + "id": "03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 86.208, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a92e16af-1756-4719-be21-9cece29bc38a", + "resource": { + "resourceType": "Observation", + "id": "a92e16af-1756-4719-be21-9cece29bc38a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 31.819, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c14633d-fd29-4ec5-9504-1c816edb6380", + "resource": { + "resourceType": "Observation", + "id": "3c14633d-fd29-4ec5-9504-1c816edb6380", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 34.785, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e075aabd-4c38-4a01-811f-b632c5485b67", + "resource": { + "resourceType": "Observation", + "id": "e075aabd-4c38-4a01-811f-b632c5485b67", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 43.731, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:684b6ca4-14a0-4b42-8c62-e22ad914963a", + "resource": { + "resourceType": "Observation", + "id": "684b6ca4-14a0-4b42-8c62-e22ad914963a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 447.75, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", + "resource": { + "resourceType": "Observation", + "id": "5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 408.89, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d25ac0a-ec16-489c-90aa-636271a2fdf1", + "resource": { + "resourceType": "Observation", + "id": "5d25ac0a-ec16-489c-90aa-636271a2fdf1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 11.461, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:88d2e17e-e828-420d-b9c8-7dc475aa93a4", + "resource": { + "resourceType": "Procedure", + "id": "88d2e17e-e828-420d-b9c8-7dc475aa93a4", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "performedPeriod": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:30:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d298345a-915e-41d5-9ba6-a8cac8e028fc", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d298345a-915e-41d5-9ba6-a8cac8e028fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:db483d27-9d6b-4a8a-8eae-822d4bc480fa", + "display": "Glucose" + }, + { + "reference": "urn:uuid:bdb81812-1403-4cc5-b70f-858f8a50984a", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:435c85d0-da18-4a27-b2c1-3c43acbffe38", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:c05fead5-b200-4676-8dbb-534a9a88837a", + "display": "Calcium" + }, + { + "reference": "urn:uuid:3b28bd42-0541-4b8a-827b-01f90e142e15", + "display": "Sodium" + }, + { + "reference": "urn:uuid:e0230357-77ff-4a2a-a241-9e6f79d8d8e6", + "display": "Potassium" + }, + { + "reference": "urn:uuid:4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", + "display": "Chloride" + }, + { + "reference": "urn:uuid:c392cac0-c1cb-4676-8620-bddfa9e59dff", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:890c6989-1e4a-4fe4-9ab5-cb1095d5437d", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:7562b88d-3f43-4089-95af-bf82c68f31d8", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4c440773-3dff-4822-a315-46fc30f099c6", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:cdde0e90-236d-4d49-b6b6-64c010a9b118", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:0213fc21-7fde-4d6b-9043-0231c3f8eac5", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:f0691777-00ab-49ce-b56d-ada1036eeab5", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:199a0c22-29cf-4e5d-9d05-3ba361fd8687", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:4223c0c9-07d7-4f8b-bd12-4276cc12f3c2", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4223c0c9-07d7-4f8b-bd12-4276cc12f3c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + }, + "effectiveDateTime": "2013-05-03T05:15:51-07:00", + "issued": "2013-05-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:5654fec6-cf82-4e31-afcf-a53507f24661", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:fb6f8e83-79cf-4508-ba06-a156a966eb8a", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:ba58aa20-515d-41e7-a436-c48af6109029", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:a92e16af-1756-4719-be21-9cece29bc38a", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:3c14633d-fd29-4ec5-9504-1c816edb6380", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:e075aabd-4c38-4a01-811f-b632c5485b67", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:684b6ca4-14a0-4b42-8c62-e22ad914963a", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:5d25ac0a-ec16-489c-90aa-636271a2fdf1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:567a83b9-9037-4a16-b72c-008a71ece928", + "resource": { + "resourceType": "Claim", + "id": "567a83b9-9037-4a16-b72c-008a71ece928", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:45:51-07:00" + }, + "created": "2013-05-03T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:88d2e17e-e828-420d-b9c8-7dc475aa93a4" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 13022.01, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:292ea288-6a0f-4352-88b0-a7ce25abcb16", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "292ea288-6a0f-4352-88b0-a7ce25abcb16", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "567a83b9-9037-4a16-b72c-008a71ece928" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-05-03T05:45:51-07:00", + "end": "2014-05-03T05:45:51-07:00" + }, + "created": "2013-05-03T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:567a83b9-9037-4a16-b72c-008a71ece928" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-05-03T05:15:51-07:00", + "end": "2013-05-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 13022.01, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2604.402, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 10417.608, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 13022.01, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 13022.01, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 10417.608, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266", + "resource": { + "resourceType": "Encounter", + "id": "25007a07-a29f-45f1-b210-46ba67fc0266", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:45:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f6f32e83-b399-4769-9e79-7b8d875ca96c", + "resource": { + "resourceType": "Observation", + "id": "f6f32e83-b399-4769-9e79-7b8d875ca96c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 8, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e10e3f7f-a4a4-40e5-b949-508a04d5a989", + "resource": { + "resourceType": "Observation", + "id": "e10e3f7f-a4a4-40e5-b949-508a04d5a989", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 89.41, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3be48981-0679-4de8-9911-ec13747379df", + "resource": { + "resourceType": "Observation", + "id": "3be48981-0679-4de8-9911-ec13747379df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 13.41, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e7da76c-9d45-431d-8551-28d544f3a66e", + "resource": { + "resourceType": "Observation", + "id": "1e7da76c-9d45-431d-8551-28d544f3a66e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.9915, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d0502848-2679-4d4b-8501-26ad0eecf9eb", + "resource": { + "resourceType": "Observation", + "id": "d0502848-2679-4d4b-8501-26ad0eecf9eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 8.56, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bd5e6353-45ac-4751-93af-beeb8255958c", + "resource": { + "resourceType": "Observation", + "id": "bd5e6353-45ac-4751-93af-beeb8255958c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 137.87, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", + "resource": { + "resourceType": "Observation", + "id": "b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.74, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", + "resource": { + "resourceType": "Observation", + "id": "fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 102.86, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", + "resource": { + "resourceType": "Observation", + "id": "7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 25.44, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b1279371-8e99-41b2-b7ff-5b3a97efe602", + "resource": { + "resourceType": "Observation", + "id": "b1279371-8e99-41b2-b7ff-5b3a97efe602", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.416, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:de5417cb-d88a-4982-9f74-b3d6e9ccb723", + "resource": { + "resourceType": "Observation", + "id": "de5417cb-d88a-4982-9f74-b3d6e9ccb723", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 76.97, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2f31c342-3e04-4ce2-b98e-2c27809c2863", + "resource": { + "resourceType": "Observation", + "id": "2f31c342-3e04-4ce2-b98e-2c27809c2863", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.2324, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7a09bece-90ba-43eb-8cbd-55a8655cb351", + "resource": { + "resourceType": "Observation", + "id": "7a09bece-90ba-43eb-8cbd-55a8655cb351", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.2795, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee676515-2265-47af-9a03-dbb01b7fa1e6", + "resource": { + "resourceType": "Observation", + "id": "ee676515-2265-47af-9a03-dbb01b7fa1e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.73116, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", + "resource": { + "resourceType": "Observation", + "id": "ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 94.61, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4006caea-a194-4c2b-a6bc-0e978cb80011", + "resource": { + "resourceType": "Observation", + "id": "4006caea-a194-4c2b-a6bc-0e978cb80011", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 28.356, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d341f336-3669-427b-ba4c-698784affc7c", + "resource": { + "resourceType": "Observation", + "id": "d341f336-3669-427b-ba4c-698784affc7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 19.842, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e9974a8-1f57-47ae-9d0d-90de9bd6b553", + "resource": { + "resourceType": "Observation", + "id": "1e9974a8-1f57-47ae-9d0d-90de9bd6b553", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 1.1343, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:376bdd74-8b7c-4c1e-865f-67c9db93c527", + "resource": { + "resourceType": "Observation", + "id": "376bdd74-8b7c-4c1e-865f-67c9db93c527", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.4919, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b44f0182-af4c-4722-9bee-e48a3458fc74", + "resource": { + "resourceType": "Observation", + "id": "b44f0182-af4c-4722-9bee-e48a3458fc74", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.033, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", + "resource": { + "resourceType": "Observation", + "id": "5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 30.417, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c383a21-1bdd-4b14-afb3-7328aefeb7c5", + "resource": { + "resourceType": "Observation", + "id": "2c383a21-1bdd-4b14-afb3-7328aefeb7c5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 82.847, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", + "resource": { + "resourceType": "Observation", + "id": "f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 29.018, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:13390f28-fb82-4110-9c9f-99a2d71adf98", + "resource": { + "resourceType": "Observation", + "id": "13390f28-fb82-4110-9c9f-99a2d71adf98", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 33.752, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", + "resource": { + "resourceType": "Observation", + "id": "7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 44.993, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:89afeeab-ffd7-4a94-a6cf-a65c81fc4727", + "resource": { + "resourceType": "Observation", + "id": "89afeeab-ffd7-4a94-a6cf-a65c81fc4727", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 161.69, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3fc4c1ed-91fa-415c-97b6-6775961ab826", + "resource": { + "resourceType": "Observation", + "id": "3fc4c1ed-91fa-415c-97b6-6775961ab826", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 184.06, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d0224e91-ce6e-499b-a927-e157b3623ff0", + "resource": { + "resourceType": "Observation", + "id": "d0224e91-ce6e-499b-a927-e157b3623ff0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 10.339, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8df22fb6-c5ee-48ed-86e0-f76f19798fb7", + "resource": { + "resourceType": "Procedure", + "id": "8df22fb6-c5ee-48ed-86e0-f76f19798fb7", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "performedPeriod": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:30:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9c166ad3-9123-4bd6-8140-d3749b2c2bae", + "resource": { + "resourceType": "DiagnosticReport", + "id": "9c166ad3-9123-4bd6-8140-d3749b2c2bae", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:e10e3f7f-a4a4-40e5-b949-508a04d5a989", + "display": "Glucose" + }, + { + "reference": "urn:uuid:3be48981-0679-4de8-9911-ec13747379df", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:1e7da76c-9d45-431d-8551-28d544f3a66e", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:d0502848-2679-4d4b-8501-26ad0eecf9eb", + "display": "Calcium" + }, + { + "reference": "urn:uuid:bd5e6353-45ac-4751-93af-beeb8255958c", + "display": "Sodium" + }, + { + "reference": "urn:uuid:b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", + "display": "Potassium" + }, + { + "reference": "urn:uuid:fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", + "display": "Chloride" + }, + { + "reference": "urn:uuid:7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:b1279371-8e99-41b2-b7ff-5b3a97efe602", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:de5417cb-d88a-4982-9f74-b3d6e9ccb723", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:2f31c342-3e04-4ce2-b98e-2c27809c2863", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:7a09bece-90ba-43eb-8cbd-55a8655cb351", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:ee676515-2265-47af-9a03-dbb01b7fa1e6", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4006caea-a194-4c2b-a6bc-0e978cb80011", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:d341f336-3669-427b-ba4c-698784affc7c", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:d5a0caa4-f0a5-488f-a1fb-75c0b2e5120f", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d5a0caa4-f0a5-488f-a1fb-75c0b2e5120f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + }, + "effectiveDateTime": "2013-06-03T05:15:51-07:00", + "issued": "2013-06-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:1e9974a8-1f57-47ae-9d0d-90de9bd6b553", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:376bdd74-8b7c-4c1e-865f-67c9db93c527", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b44f0182-af4c-4722-9bee-e48a3458fc74", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:2c383a21-1bdd-4b14-afb3-7328aefeb7c5", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:13390f28-fb82-4110-9c9f-99a2d71adf98", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:89afeeab-ffd7-4a94-a6cf-a65c81fc4727", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:3fc4c1ed-91fa-415c-97b6-6775961ab826", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:d0224e91-ce6e-499b-a927-e157b3623ff0", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa", + "resource": { + "resourceType": "Claim", + "id": "6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:45:51-07:00" + }, + "created": "2013-06-03T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:8df22fb6-c5ee-48ed-86e0-f76f19798fb7" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 13305.82, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:84114e2d-ea86-4f2d-9f76-e8fdc2a83693", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "84114e2d-ea86-4f2d-9f76-e8fdc2a83693", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-06-03T05:45:51-07:00", + "end": "2014-06-03T05:45:51-07:00" + }, + "created": "2013-06-03T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-06-03T05:15:51-07:00", + "end": "2013-06-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 13305.82, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2661.164, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 10644.656, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 13305.82, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 13305.82, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 10644.656, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320", + "resource": { + "resourceType": "Encounter", + "id": "7009e93b-a80f-4c2e-98c0-450dcb2ee320", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:45:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d4fb7920-826e-412f-a0b2-434698867a36", + "resource": { + "resourceType": "Observation", + "id": "d4fb7920-826e-412f-a0b2-434698867a36", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 6, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a4ea7bc1-0ace-484f-8fff-d447cca6a288", + "resource": { + "resourceType": "Observation", + "id": "a4ea7bc1-0ace-484f-8fff-d447cca6a288", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 93.35, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50263b45-98e1-4f66-96ea-b99009163f88", + "resource": { + "resourceType": "Observation", + "id": "50263b45-98e1-4f66-96ea-b99009163f88", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 12.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed4b9588-f6d7-4a39-9aef-dcc51d087055", + "resource": { + "resourceType": "Observation", + "id": "ed4b9588-f6d7-4a39-9aef-dcc51d087055", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.9351, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4db612bc-9da6-4d3d-b7db-fff64e0df3e9", + "resource": { + "resourceType": "Observation", + "id": "4db612bc-9da6-4d3d-b7db-fff64e0df3e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.82, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:055632d4-9655-4066-aa4f-e7bc8962c5cc", + "resource": { + "resourceType": "Observation", + "id": "055632d4-9655-4066-aa4f-e7bc8962c5cc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 141.37, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6bbb3f02-95df-40b3-8a9c-d4dd59246994", + "resource": { + "resourceType": "Observation", + "id": "6bbb3f02-95df-40b3-8a9c-d4dd59246994", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.14, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9abe6e75-09f1-44f1-b141-7aa01c9c54e0", + "resource": { + "resourceType": "Observation", + "id": "9abe6e75-09f1-44f1-b141-7aa01c9c54e0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 102.1, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11c98316-11f0-4215-ba70-ae58507353b1", + "resource": { + "resourceType": "Observation", + "id": "11c98316-11f0-4215-ba70-ae58507353b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 23.18, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c8b1ccb1-c19b-40a3-af13-3a54e6e06397", + "resource": { + "resourceType": "Observation", + "id": "c8b1ccb1-c19b-40a3-af13-3a54e6e06397", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 7.9529, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", + "resource": { + "resourceType": "Observation", + "id": "01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 76.827, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:91975c4e-4ddb-4774-a317-35cb425ae7eb", + "resource": { + "resourceType": "Observation", + "id": "91975c4e-4ddb-4774-a317-35cb425ae7eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.9537, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e16012c-f25a-4ccd-bcbc-3f8064f5284d", + "resource": { + "resourceType": "Observation", + "id": "4e16012c-f25a-4ccd-bcbc-3f8064f5284d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.9078, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5901e8e5-a877-4545-afa9-963dea832a63", + "resource": { + "resourceType": "Observation", + "id": "5901e8e5-a877-4545-afa9-963dea832a63", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.29781, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:da2381dd-5766-4a49-b30c-3c4f81dd4729", + "resource": { + "resourceType": "Observation", + "id": "da2381dd-5766-4a49-b30c-3c4f81dd4729", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 56.614, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", + "resource": { + "resourceType": "Observation", + "id": "4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 57.996, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:73e22621-70c6-4862-a5e7-2ee5c133b1aa", + "resource": { + "resourceType": "Observation", + "id": "73e22621-70c6-4862-a5e7-2ee5c133b1aa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 26.657, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2f7908cb-b33d-409f-a209-6c42835c6fe4", + "resource": { + "resourceType": "Observation", + "id": "2f7908cb-b33d-409f-a209-6c42835c6fe4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.6457, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cdcc9f05-f61a-4a96-8f91-4cc18e15758d", + "resource": { + "resourceType": "Observation", + "id": "cdcc9f05-f61a-4a96-8f91-4cc18e15758d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 5.4292, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c44df0f8-e1d7-4ec7-9a53-a5c78616f455", + "resource": { + "resourceType": "Observation", + "id": "c44df0f8-e1d7-4ec7-9a53-a5c78616f455", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 12.151, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:08bebef2-cab9-4d3c-989b-bdc5c6719a79", + "resource": { + "resourceType": "Observation", + "id": "08bebef2-cab9-4d3c-989b-bdc5c6719a79", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 32.333, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0257b1a-b770-4d89-9221-ead5e073ed09", + "resource": { + "resourceType": "Observation", + "id": "b0257b1a-b770-4d89-9221-ead5e073ed09", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 92.156, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1234710d-7995-4d8b-a6c6-f30354d3ef15", + "resource": { + "resourceType": "Observation", + "id": "1234710d-7995-4d8b-a6c6-f30354d3ef15", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 29.12, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9d2361d-756a-4fef-8b1e-c570d04f8e12", + "resource": { + "resourceType": "Observation", + "id": "a9d2361d-756a-4fef-8b1e-c570d04f8e12", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 34.547, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3ea2b6fc-982b-464d-8b7d-1c3c124c5192", + "resource": { + "resourceType": "Observation", + "id": "3ea2b6fc-982b-464d-8b7d-1c3c124c5192", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 40.081, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4f51dfab-e10b-40fb-ba72-993732dd716c", + "resource": { + "resourceType": "Observation", + "id": "4f51dfab-e10b-40fb-ba72-993732dd716c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 196.26, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c65c3db-8493-4dba-a427-8ccbd3d01aa4", + "resource": { + "resourceType": "Observation", + "id": "1c65c3db-8493-4dba-a427-8ccbd3d01aa4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 381.15, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:89d4bb3b-63df-45ce-9997-facc4934fb49", + "resource": { + "resourceType": "Observation", + "id": "89d4bb3b-63df-45ce-9997-facc4934fb49", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "valueQuantity": { + "value": 12.3, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8301b201-36f6-4076-ad21-33fb2e23fe95", + "resource": { + "resourceType": "Procedure", + "id": "8301b201-36f6-4076-ad21-33fb2e23fe95", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "performedPeriod": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:30:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d7d87227-51f3-4ac0-82f8-1d4f1daf5346", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d7d87227-51f3-4ac0-82f8-1d4f1daf5346", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:a4ea7bc1-0ace-484f-8fff-d447cca6a288", + "display": "Glucose" + }, + { + "reference": "urn:uuid:50263b45-98e1-4f66-96ea-b99009163f88", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:ed4b9588-f6d7-4a39-9aef-dcc51d087055", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:4db612bc-9da6-4d3d-b7db-fff64e0df3e9", + "display": "Calcium" + }, + { + "reference": "urn:uuid:055632d4-9655-4066-aa4f-e7bc8962c5cc", + "display": "Sodium" + }, + { + "reference": "urn:uuid:6bbb3f02-95df-40b3-8a9c-d4dd59246994", + "display": "Potassium" + }, + { + "reference": "urn:uuid:9abe6e75-09f1-44f1-b141-7aa01c9c54e0", + "display": "Chloride" + }, + { + "reference": "urn:uuid:11c98316-11f0-4215-ba70-ae58507353b1", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:c8b1ccb1-c19b-40a3-af13-3a54e6e06397", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:91975c4e-4ddb-4774-a317-35cb425ae7eb", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4e16012c-f25a-4ccd-bcbc-3f8064f5284d", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:5901e8e5-a877-4545-afa9-963dea832a63", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:da2381dd-5766-4a49-b30c-3c4f81dd4729", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:73e22621-70c6-4862-a5e7-2ee5c133b1aa", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f199e05c-8fca-424d-8725-d1e23528b1e0", + "resource": { + "resourceType": "DiagnosticReport", + "id": "f199e05c-8fca-424d-8725-d1e23528b1e0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + }, + "effectiveDateTime": "2013-07-01T05:15:51-07:00", + "issued": "2013-07-01T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:2f7908cb-b33d-409f-a209-6c42835c6fe4", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:cdcc9f05-f61a-4a96-8f91-4cc18e15758d", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:c44df0f8-e1d7-4ec7-9a53-a5c78616f455", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:08bebef2-cab9-4d3c-989b-bdc5c6719a79", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:b0257b1a-b770-4d89-9221-ead5e073ed09", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:1234710d-7995-4d8b-a6c6-f30354d3ef15", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:a9d2361d-756a-4fef-8b1e-c570d04f8e12", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:3ea2b6fc-982b-464d-8b7d-1c3c124c5192", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:4f51dfab-e10b-40fb-ba72-993732dd716c", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1c65c3db-8493-4dba-a427-8ccbd3d01aa4", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:89d4bb3b-63df-45ce-9997-facc4934fb49", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0ec20549-3f16-4ae2-ba60-569c3f80ad65", + "resource": { + "resourceType": "Claim", + "id": "0ec20549-3f16-4ae2-ba60-569c3f80ad65", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:45:51-07:00" + }, + "created": "2013-07-01T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:8301b201-36f6-4076-ad21-33fb2e23fe95" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 9589.76, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6fc952e0-1cb7-4b84-ba46-22ce2056e59b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6fc952e0-1cb7-4b84-ba46-22ce2056e59b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0ec20549-3f16-4ae2-ba60-569c3f80ad65" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-07-01T05:45:51-07:00", + "end": "2014-07-01T05:45:51-07:00" + }, + "created": "2013-07-01T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0ec20549-3f16-4ae2-ba60-569c3f80ad65" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-07-01T05:15:51-07:00", + "end": "2013-07-01T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 9589.76, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1917.9520000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7671.808000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 9589.76, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 9589.76, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7671.808000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69", + "resource": { + "resourceType": "Encounter", + "id": "ee0032a1-183c-4106-a058-b530e2e8df69", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:45:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:45:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c1af820a-72e4-4b68-8a08-67674f95156b", + "resource": { + "resourceType": "Observation", + "id": "c1af820a-72e4-4b68-8a08-67674f95156b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 7, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3668a3c-b125-4c1f-bcb9-2718c969bda1", + "resource": { + "resourceType": "Observation", + "id": "e3668a3c-b125-4c1f-bcb9-2718c969bda1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 86.96, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3100271-6dab-4757-8162-33aa845b2be1", + "resource": { + "resourceType": "Observation", + "id": "e3100271-6dab-4757-8162-33aa845b2be1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 15.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9e47b447-6e7e-464c-9130-8a9459f67335", + "resource": { + "resourceType": "Observation", + "id": "9e47b447-6e7e-464c-9130-8a9459f67335", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.5959, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0275af97-f3dc-44ac-a196-a0427c5130f4", + "resource": { + "resourceType": "Observation", + "id": "0275af97-f3dc-44ac-a196-a0427c5130f4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 9.32, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7ac76bed-dd47-4e2d-bb84-3b4e884d3277", + "resource": { + "resourceType": "Observation", + "id": "7ac76bed-dd47-4e2d-bb84-3b4e884d3277", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 139.22, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:594abc12-1f5d-4514-b14f-f55fec4a1cc2", + "resource": { + "resourceType": "Observation", + "id": "594abc12-1f5d-4514-b14f-f55fec4a1cc2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.65, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c712bc3-0b86-4b38-bfa8-5696add93de2", + "resource": { + "resourceType": "Observation", + "id": "0c712bc3-0b86-4b38-bfa8-5696add93de2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 108.64, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34afb9aa-f874-493b-b286-4319b1e1f950", + "resource": { + "resourceType": "Observation", + "id": "34afb9aa-f874-493b-b286-4319b1e1f950", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 25.89, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bf9e0a34-aa41-4d98-b69a-4d9a3762e349", + "resource": { + "resourceType": "Observation", + "id": "bf9e0a34-aa41-4d98-b69a-4d9a3762e349", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33914-3", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + } + ], + "text": "Glomerular filtration rate/1.73 sq M.predicted" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 23.513, + "unit": "mL/min", + "system": "http://unitsofmeasure.org", + "code": "mL/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2f5ed8d-9778-4fe6-a38a-0d08becb519a", + "resource": { + "resourceType": "Observation", + "id": "e2f5ed8d-9778-4fe6-a38a-0d08becb519a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2885-2", + "display": "Protein [Mass/volume] in Serum or Plasma" + } + ], + "text": "Protein [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 71.487, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", + "resource": { + "resourceType": "Observation", + "id": "4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1751-7", + "display": "Albumin [Mass/volume] in Serum or Plasma" + } + ], + "text": "Albumin [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 3.5381, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:950b45e8-69e9-4a4e-b934-38790b46e387", + "resource": { + "resourceType": "Observation", + "id": "950b45e8-69e9-4a4e-b934-38790b46e387", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "10834-0", + "display": "Globulin [Mass/volume] in Serum by calculation" + } + ], + "text": "Globulin [Mass/volume] in Serum by calculation" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.8078, + "unit": "g/L", + "system": "http://unitsofmeasure.org", + "code": "g/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:90d66f1a-7eaf-40b8-ac33-b3a65956ec11", + "resource": { + "resourceType": "Observation", + "id": "90d66f1a-7eaf-40b8-ac33-b3a65956ec11", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1975-2", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + } + ], + "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 0.3802, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:901f8064-d960-444f-acc8-020b767f39f6", + "resource": { + "resourceType": "Observation", + "id": "901f8064-d960-444f-acc8-020b767f39f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6768-6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 136.15, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25336e04-d01b-4aa8-9ee9-2daed53b3380", + "resource": { + "resourceType": "Observation", + "id": "25336e04-d01b-4aa8-9ee9-2daed53b3380", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1742-6", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 20.644, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:57c183ce-8062-4059-8fb9-230cbb45dace", + "resource": { + "resourceType": "Observation", + "id": "57c183ce-8062-4059-8fb9-230cbb45dace", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "1920-8", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ], + "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 14.904, + "unit": "U/L", + "system": "http://unitsofmeasure.org", + "code": "U/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4a1bfb0-21f1-4731-b150-56287fe9e8d2", + "resource": { + "resourceType": "Observation", + "id": "d4a1bfb0-21f1-4731-b150-56287fe9e8d2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 2.483, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e919c3e-abd3-4f1b-8d19-51b6b305eda7", + "resource": { + "resourceType": "Observation", + "id": "7e919c3e-abd3-4f1b-8d19-51b6b305eda7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 4.4587, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c86cf3ba-5cc2-4987-9525-25d7cb3960ec", + "resource": { + "resourceType": "Observation", + "id": "c86cf3ba-5cc2-4987-9525-25d7cb3960ec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 11.79, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:61582809-9b25-4cf7-9601-c052baefa448", + "resource": { + "resourceType": "Observation", + "id": "61582809-9b25-4cf7-9601-c052baefa448", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 30.912, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:637b3bc4-a0d4-4991-bae6-d0844683d041", + "resource": { + "resourceType": "Observation", + "id": "637b3bc4-a0d4-4991-bae6-d0844683d041", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 88.162, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ea779f6a-ea11-4bb0-a479-8949b497b137", + "resource": { + "resourceType": "Observation", + "id": "ea779f6a-ea11-4bb0-a479-8949b497b137", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 30.265, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0beca38-819a-4d66-b2e1-91f0dee928e0", + "resource": { + "resourceType": "Observation", + "id": "c0beca38-819a-4d66-b2e1-91f0dee928e0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 34.142, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:39678dda-38d6-44d2-b4c0-b187d8054845", + "resource": { + "resourceType": "Observation", + "id": "39678dda-38d6-44d2-b4c0-b187d8054845", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 41.58, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e14abdb0-87de-425f-b84a-6921fd094428", + "resource": { + "resourceType": "Observation", + "id": "e14abdb0-87de-425f-b84a-6921fd094428", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 188.57, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:88868082-4d03-43ff-b9a8-c20fd8a3d807", + "resource": { + "resourceType": "Observation", + "id": "88868082-4d03-43ff-b9a8-c20fd8a3d807", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 357.31, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cc38aa2-262d-40f2-bb59-fc288edfd0fe", + "resource": { + "resourceType": "Observation", + "id": "0cc38aa2-262d-40f2-bb59-fc288edfd0fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "valueQuantity": { + "value": 11.371, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4fee3798-31b4-4cb9-8bc7-1e8118c962c4", + "resource": { + "resourceType": "Procedure", + "id": "4fee3798-31b4-4cb9-8bc7-1e8118c962c4", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "performedPeriod": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:30:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", + "display": "Overlapping malignant neoplasm of colon" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e576d614-822a-43b9-b66a-993e451bee8c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "e576d614-822a-43b9-b66a-993e451bee8c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "24323-8", + "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" + } + ], + "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:e3668a3c-b125-4c1f-bcb9-2718c969bda1", + "display": "Glucose" + }, + { + "reference": "urn:uuid:e3100271-6dab-4757-8162-33aa845b2be1", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:9e47b447-6e7e-464c-9130-8a9459f67335", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:0275af97-f3dc-44ac-a196-a0427c5130f4", + "display": "Calcium" + }, + { + "reference": "urn:uuid:7ac76bed-dd47-4e2d-bb84-3b4e884d3277", + "display": "Sodium" + }, + { + "reference": "urn:uuid:594abc12-1f5d-4514-b14f-f55fec4a1cc2", + "display": "Potassium" + }, + { + "reference": "urn:uuid:0c712bc3-0b86-4b38-bfa8-5696add93de2", + "display": "Chloride" + }, + { + "reference": "urn:uuid:34afb9aa-f874-493b-b286-4319b1e1f950", + "display": "Carbon Dioxide" + }, + { + "reference": "urn:uuid:bf9e0a34-aa41-4d98-b69a-4d9a3762e349", + "display": "Glomerular filtration rate/1.73 sq M.predicted" + }, + { + "reference": "urn:uuid:e2f5ed8d-9778-4fe6-a38a-0d08becb519a", + "display": "Protein [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", + "display": "Albumin [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:950b45e8-69e9-4a4e-b934-38790b46e387", + "display": "Globulin [Mass/volume] in Serum by calculation" + }, + { + "reference": "urn:uuid:90d66f1a-7eaf-40b8-ac33-b3a65956ec11", + "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:901f8064-d960-444f-acc8-020b767f39f6", + "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:25336e04-d01b-4aa8-9ee9-2daed53b3380", + "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + }, + { + "reference": "urn:uuid:57c183ce-8062-4059-8fb9-230cbb45dace", + "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:83d2fa1e-edd6-47d5-99e8-1571d1f39f4c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "83d2fa1e-edd6-47d5-99e8-1571d1f39f4c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + }, + "effectiveDateTime": "2013-08-03T05:15:51-07:00", + "issued": "2013-08-03T05:15:51.927-07:00", + "result": [ + { + "reference": "urn:uuid:d4a1bfb0-21f1-4731-b150-56287fe9e8d2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:7e919c3e-abd3-4f1b-8d19-51b6b305eda7", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:c86cf3ba-5cc2-4987-9525-25d7cb3960ec", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:61582809-9b25-4cf7-9601-c052baefa448", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:637b3bc4-a0d4-4991-bae6-d0844683d041", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:ea779f6a-ea11-4bb0-a479-8949b497b137", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:c0beca38-819a-4d66-b2e1-91f0dee928e0", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:39678dda-38d6-44d2-b4c0-b187d8054845", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:e14abdb0-87de-425f-b84a-6921fd094428", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:88868082-4d03-43ff-b9a8-c20fd8a3d807", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:0cc38aa2-262d-40f2-bb59-fc288edfd0fe", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030", + "resource": { + "resourceType": "Claim", + "id": "b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:45:51-07:00" + }, + "created": "2013-08-03T05:45:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4fee3798-31b4-4cb9-8bc7-1e8118c962c4" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "net": { + "value": 16455.78, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3add92b6-2b88-42c3-9f6c-3fd5dbe37a04", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3add92b6-2b88-42c3-9f6c-3fd5dbe37a04", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-08-03T05:45:51-07:00", + "end": "2014-08-03T05:45:51-07:00" + }, + "created": "2013-08-03T05:45:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703423002", + "display": "Combined chemotherapy and radiation therapy (procedure)" + } + ], + "text": "Combined chemotherapy and radiation therapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-08-03T05:15:51-07:00", + "end": "2013-08-03T05:45:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 16455.78, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 3291.156, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 13164.624, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 16455.78, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 16455.78, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 13164.624, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455", + "resource": { + "resourceType": "Encounter", + "id": "5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:64ffad44-b085-4412-a9be-f775e0786881", + "resource": { + "resourceType": "Observation", + "id": "64ffad44-b085-4412-a9be-f775e0786881", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d9b8422-c3c6-4a8c-9853-ee19f0565a33", + "resource": { + "resourceType": "Observation", + "id": "3d9b8422-c3c6-4a8c-9853-ee19f0565a33", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:54b8328f-9ab2-4852-998a-6a0853292537", + "resource": { + "resourceType": "Observation", + "id": "54b8328f-9ab2-4852-998a-6a0853292537", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:081942d4-b592-411c-9e0b-d00b317de99b", + "resource": { + "resourceType": "Observation", + "id": "081942d4-b592-411c-9e0b-d00b317de99b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e878ed4-644f-449c-91c0-a69df02bb0eb", + "resource": { + "resourceType": "Observation", + "id": "6e878ed4-644f-449c-91c0-a69df02bb0eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 122, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa1ffeab-e59f-47ea-b09e-1848ab36656b", + "resource": { + "resourceType": "Observation", + "id": "fa1ffeab-e59f-47ea-b09e-1848ab36656b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 65, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c11ca22-4d01-44f5-9e63-0f77ccc96790", + "resource": { + "resourceType": "Observation", + "id": "0c11ca22-4d01-44f5-9e63-0f77ccc96790", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bd4b198f-5e03-44ce-978c-ee474501fc5f", + "resource": { + "resourceType": "Observation", + "id": "bd4b198f-5e03-44ce-978c-ee474501fc5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 72.76, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:01c7a467-8130-4c85-9a6a-c5cd25140ccb", + "resource": { + "resourceType": "Observation", + "id": "01c7a467-8130-4c85-9a6a-c5cd25140ccb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 10.73, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9abdda58-25c5-42b9-a379-2b7b440184c1", + "resource": { + "resourceType": "Observation", + "id": "9abdda58-25c5-42b9-a379-2b7b440184c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.71, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0f7854fc-dec6-4d76-8c66-6e2f512da711", + "resource": { + "resourceType": "Observation", + "id": "0f7854fc-dec6-4d76-8c66-6e2f512da711", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 8.59, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3e46da58-5a10-4449-8a72-039c0e325b27", + "resource": { + "resourceType": "Observation", + "id": "3e46da58-5a10-4449-8a72-039c0e325b27", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 141.62, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35cef2f5-8c00-4c9f-a524-0c653f2a9a73", + "resource": { + "resourceType": "Observation", + "id": "35cef2f5-8c00-4c9f-a524-0c653f2a9a73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 3.81, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:395b3672-b878-4e1e-a582-c29cf8915688", + "resource": { + "resourceType": "Observation", + "id": "395b3672-b878-4e1e-a582-c29cf8915688", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 110.09, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e636cdd-9913-4985-a409-aaa700fc07d6", + "resource": { + "resourceType": "Observation", + "id": "6e636cdd-9913-4985-a409-aaa700fc07d6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.04, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5b8e582a-ec17-493e-a837-170cdab88510", + "resource": { + "resourceType": "Observation", + "id": "5b8e582a-ec17-493e-a837-170cdab88510", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 197.31, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e73dd94a-0eb9-4077-9a21-06c0c8591b6b", + "resource": { + "resourceType": "Observation", + "id": "e73dd94a-0eb9-4077-9a21-06c0c8591b6b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 120.68, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8fb5417-cd77-44b9-94a3-72dbe33268a9", + "resource": { + "resourceType": "Observation", + "id": "d8fb5417-cd77-44b9-94a3-72dbe33268a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 105.69, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9309f308-a016-44bd-993d-53b024c8e6d6", + "resource": { + "resourceType": "Observation", + "id": "9309f308-a016-44bd-993d-53b024c8e6d6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 67.48, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5997de0b-7dac-4b65-973b-13e48a70e242", + "resource": { + "resourceType": "Observation", + "id": "5997de0b-7dac-4b65-973b-13e48a70e242", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.6333, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", + "resource": { + "resourceType": "Observation", + "id": "8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.8747, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65f5c466-b967-4a25-8b30-46aa7f632247", + "resource": { + "resourceType": "Observation", + "id": "65f5c466-b967-4a25-8b30-46aa7f632247", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 17.009, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:214ce259-d519-4273-9e68-7cf6216cbd7c", + "resource": { + "resourceType": "Observation", + "id": "214ce259-d519-4273-9e68-7cf6216cbd7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 43.181, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fffe3446-76bd-451b-8613-44811836bef8", + "resource": { + "resourceType": "Observation", + "id": "fffe3446-76bd-451b-8613-44811836bef8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 90.459, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1701017-6395-429d-afa6-baa7442b690f", + "resource": { + "resourceType": "Observation", + "id": "a1701017-6395-429d-afa6-baa7442b690f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.263, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e4335d63-a06f-4955-b199-74e791efbf54", + "resource": { + "resourceType": "Observation", + "id": "e4335d63-a06f-4955-b199-74e791efbf54", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 33.812, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", + "resource": { + "resourceType": "Observation", + "id": "a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 43.011, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83ecfdd5-9879-41c1-80bb-f929cabf55e1", + "resource": { + "resourceType": "Observation", + "id": "83ecfdd5-9879-41c1-80bb-f929cabf55e1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 210.45, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:022dd7d5-86e4-4736-a64d-d46b5a142983", + "resource": { + "resourceType": "Observation", + "id": "022dd7d5-86e4-4736-a64d-d46b5a142983", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 238.39, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e993cda-6614-4dda-b26b-1b39e3e66feb", + "resource": { + "resourceType": "Observation", + "id": "6e993cda-6614-4dda-b26b-1b39e3e66feb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 11.506, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c5538e6d-377f-4a1a-bf79-d7087505d02f", + "resource": { + "resourceType": "Observation", + "id": "c5538e6d-377f-4a1a-bf79-d7087505d02f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:944eb7a4-9849-4bd8-9897-63af2f11366c", + "resource": { + "resourceType": "Observation", + "id": "944eb7a4-9849-4bd8-9897-63af2f11366c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.14, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2726655b-6af7-4bb7-a5ca-c0d51b9b86ef", + "resource": { + "resourceType": "Procedure", + "id": "2726655b-6af7-4bb7-a5ca-c0d51b9b86ef", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "performedPeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:30:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:640affe1-0ab4-4e50-bca0-1d1d30b3433d", + "resource": { + "resourceType": "MedicationRequest", + "id": "640affe1-0ab4-4e50-bca0-1d1d30b3433d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "authoredOn": "2013-11-04T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:adc24942-4e53-4fd3-877e-c5d06ca25bc7", + "resource": { + "resourceType": "Claim", + "id": "adc24942-4e53-4fd3-877e-c5d06ca25bc7", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "created": "2013-11-04T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:640affe1-0ab4-4e50-bca0-1d1d30b3433d" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:66ba38cf-4f6f-4a75-8859-e646409be000", + "resource": { + "resourceType": "Immunization", + "id": "66ba38cf-4f6f-4a75-8859-e646409be000", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "occurrenceDateTime": "2013-11-04T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:749ca395-43c7-4f67-b6f4-9b805343b108", + "resource": { + "resourceType": "Immunization", + "id": "749ca395-43c7-4f67-b6f4-9b805343b108", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "33", + "display": "pneumococcal polysaccharide vaccine, 23 valent" + } + ], + "text": "pneumococcal polysaccharide vaccine, 23 valent" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "occurrenceDateTime": "2013-11-04T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:61e8b09a-d2e5-4a5e-a979-8ce1e1d2a86c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "61e8b09a-d2e5-4a5e-a979-8ce1e1d2a86c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:bd4b198f-5e03-44ce-978c-ee474501fc5f", + "display": "Glucose" + }, + { + "reference": "urn:uuid:01c7a467-8130-4c85-9a6a-c5cd25140ccb", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:9abdda58-25c5-42b9-a379-2b7b440184c1", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:0f7854fc-dec6-4d76-8c66-6e2f512da711", + "display": "Calcium" + }, + { + "reference": "urn:uuid:3e46da58-5a10-4449-8a72-039c0e325b27", + "display": "Sodium" + }, + { + "reference": "urn:uuid:35cef2f5-8c00-4c9f-a524-0c653f2a9a73", + "display": "Potassium" + }, + { + "reference": "urn:uuid:395b3672-b878-4e1e-a582-c29cf8915688", + "display": "Chloride" + }, + { + "reference": "urn:uuid:6e636cdd-9913-4985-a409-aaa700fc07d6", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:6d148349-8907-48f4-9011-f02be63deea9", + "resource": { + "resourceType": "DiagnosticReport", + "id": "6d148349-8907-48f4-9011-f02be63deea9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:5b8e582a-ec17-493e-a837-170cdab88510", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:e73dd94a-0eb9-4077-9a21-06c0c8591b6b", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:d8fb5417-cd77-44b9-94a3-72dbe33268a9", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:9309f308-a016-44bd-993d-53b024c8e6d6", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:6e2d9695-c690-44b6-80f8-a552dbee6d07", + "resource": { + "resourceType": "DiagnosticReport", + "id": "6e2d9695-c690-44b6-80f8-a552dbee6d07", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + }, + "effectiveDateTime": "2013-11-04T04:15:51-08:00", + "issued": "2013-11-04T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:5997de0b-7dac-4b65-973b-13e48a70e242", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:65f5c466-b967-4a25-8b30-46aa7f632247", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:214ce259-d519-4273-9e68-7cf6216cbd7c", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:fffe3446-76bd-451b-8613-44811836bef8", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:a1701017-6395-429d-afa6-baa7442b690f", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:e4335d63-a06f-4955-b199-74e791efbf54", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:83ecfdd5-9879-41c1-80bb-f929cabf55e1", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:022dd7d5-86e4-4736-a64d-d46b5a142983", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:6e993cda-6614-4dda-b26b-1b39e3e66feb", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:95f72712-d5b1-4050-af79-829974f2b7cc", + "resource": { + "resourceType": "Claim", + "id": "95f72712-d5b1-4050-af79-829974f2b7cc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "created": "2013-11-04T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:66ba38cf-4f6f-4a75-8859-e646409be000" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:749ca395-43c7-4f67-b6f4-9b805343b108" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:2726655b-6af7-4bb7-a5ca-c0d51b9b86ef" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "33", + "display": "pneumococcal polysaccharide vaccine, 23 valent" + } + ], + "text": "pneumococcal polysaccharide vaccine, 23 valent" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 456.60, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d5b1fb1d-a0cf-4659-834a-cb9066c375c0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d5b1fb1d-a0cf-4659-834a-cb9066c375c0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "95f72712-d5b1-4050-af79-829974f2b7cc" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-11-04T04:45:51-08:00", + "end": "2014-11-04T04:45:51-08:00" + }, + "created": "2013-11-04T04:45:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:95f72712-d5b1-4050-af79-829974f2b7cc" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "33", + "display": "pneumococcal polysaccharide vaccine, 23 valent" + } + ], + "text": "pneumococcal polysaccharide vaccine, 23 valent" + }, + "servicedPeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-04T04:15:51-08:00", + "end": "2013-11-04T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 456.60, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 91.32000000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 365.28000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 456.60, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 456.60, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 590.1120000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441", + "resource": { + "resourceType": "Encounter", + "id": "b14cf9f8-7078-4d99-8753-417c656a7441", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "86013001", + "display": "Periodic reevaluation and management of healthy individual (procedure)" + } + ], + "text": "Periodic reevaluation and management of healthy individual (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "resource": { + "resourceType": "Condition", + "id": "89830890-3c8f-4911-8f8d-454837614455", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ], + "text": "Malignant neoplasm of breast (disorder)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "onsetDateTime": "2013-11-22T04:15:51-08:00", + "recordedDate": "2013-11-22T04:15:51-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:0c61e1c3-148a-4d6f-aec7-cb83bebe4300", + "resource": { + "resourceType": "Observation", + "id": "0c61e1c3-148a-4d6f-aec7-cb83bebe4300", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "44667-4", + "display": "Site of distant metastasis in Breast tumor" + } + ], + "text": "Site of distant metastasis in Breast tumor" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "260413007", + "display": "None (qualifier value)" + } + ], + "text": "None (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:608c342f-3431-438e-b8a2-4bed67b9e504", + "resource": { + "resourceType": "Observation", + "id": "608c342f-3431-438e-b8a2-4bed67b9e504", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21907-1", + "display": "Distant metastases.clinical [Class] Cancer" + } + ], + "text": "Distant metastases.clinical [Class] Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "30893008", + "display": "M0 category (finding)" + } + ], + "text": "M0 category (finding)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c657df3f-48ef-4f6b-b190-5fb6707f7e85", + "resource": { + "resourceType": "Observation", + "id": "c657df3f-48ef-4f6b-b190-5fb6707f7e85", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "33728-7", + "display": "Size.maximum dimension in Tumor" + } + ], + "text": "Size.maximum dimension in Tumor" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.022949, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1824225b-c940-476e-a2da-030dc2eeeaad", + "resource": { + "resourceType": "Observation", + "id": "1824225b-c940-476e-a2da-030dc2eeeaad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21905-5", + "display": "Primary tumor.clinical [Class] Cancer" + } + ], + "text": "Primary tumor.clinical [Class] Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23351008", + "display": "T1 category (finding)" + } + ], + "text": "T1 category (finding)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b59905f1-ad49-46cd-9257-d3b907d14a12", + "resource": { + "resourceType": "Observation", + "id": "b59905f1-ad49-46cd-9257-d3b907d14a12", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85352-3", + "display": "Lymph nodes with isolated tumor cells [#] in Cancer specimen by Light microscopy" + } + ], + "text": "Lymph nodes with isolated tumor cells [#] in Cancer specimen by Light microscopy" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueQuantity": { + "value": 0, + "unit": "#", + "system": "http://unitsofmeasure.org", + "code": "#" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b40588a2-94a4-4857-b305-153c711715e6", + "resource": { + "resourceType": "Observation", + "id": "b40588a2-94a4-4857-b305-153c711715e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "imaging", + "display": "imaging" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21906-3", + "display": "Regional lymph nodes.clinical [Class] Cancer" + } + ], + "text": "Regional lymph nodes.clinical [Class] Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "62455006", + "display": "N0 category (finding)" + } + ], + "text": "N0 category (finding)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2d03aae-348e-4615-b33e-f460985ad708", + "resource": { + "resourceType": "Observation", + "id": "a2d03aae-348e-4615-b33e-f460985ad708", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85319-2", + "display": "HER2 [Presence] in Breast cancer specimen by Immune stain" + } + ], + "text": "HER2 [Presence] in Breast cancer specimen by Immune stain" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "260385009", + "display": "Negative (qualifier value)" + } + ], + "text": "Negative (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d69bc4f6-2881-449d-9f25-9dfa218f559c", + "resource": { + "resourceType": "Observation", + "id": "d69bc4f6-2881-449d-9f25-9dfa218f559c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85318-4", + "display": "HER2 [Presence] in Breast cancer specimen by FISH" + } + ], + "text": "HER2 [Presence] in Breast cancer specimen by FISH" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueString": "less than 1.8" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:234a71bd-87ca-431e-9e07-f7d9433a9c60", + "resource": { + "resourceType": "Observation", + "id": "234a71bd-87ca-431e-9e07-f7d9433a9c60", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85337-4", + "display": "Estrogen receptor Ag [Presence] in Breast cancer specimen by Immune stain" + } + ], + "text": "Estrogen receptor Ag [Presence] in Breast cancer specimen by Immune stain" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "260385009", + "display": "Negative (qualifier value)" + } + ], + "text": "Negative (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:17d32508-2449-44dc-8829-1e371bfb905e", + "resource": { + "resourceType": "Observation", + "id": "17d32508-2449-44dc-8829-1e371bfb905e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85339-0", + "display": "Progesterone receptor Ag [Presence] in Breast cancer specimen by Immune stain" + } + ], + "text": "Progesterone receptor Ag [Presence] in Breast cancer specimen by Immune stain" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10828004", + "display": "Positive (qualifier value)" + } + ], + "text": "Positive (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e07d6bd-9d81-463d-968f-6ab4243d11b9", + "resource": { + "resourceType": "Observation", + "id": "6e07d6bd-9d81-463d-968f-6ab4243d11b9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21908-9", + "display": "Stage group.clinical Cancer" + } + ], + "text": "Stage group.clinical Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "261634002", + "display": "Stage 1A (qualifier value)" + } + ], + "text": "Stage 1A (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f2a9021-0944-49de-810c-0207545d5cc4", + "resource": { + "resourceType": "Observation", + "id": "1f2a9021-0944-49de-810c-0207545d5cc4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21908-9", + "display": "Stage group.clinical Cancer" + } + ], + "text": "Stage group.clinical Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "effectiveDateTime": "2013-11-22T04:15:51-08:00", + "issued": "2013-11-22T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "258215001", + "display": "Stage 1 (qualifier value)" + } + ], + "text": "Stage 1 (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d9bcd92-efb2-4ec2-8e94-2960f2118652", + "resource": { + "resourceType": "Procedure", + "id": "2d9bcd92-efb2-4ec2-8e94-2960f2118652", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "24623002", + "display": "Screening mammography (procedure)" + } + ], + "text": "Screening mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "performedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-22T04:35:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7bdf4525-68b7-42a1-8f40-c5f5bbbfab70", + "resource": { + "resourceType": "Procedure", + "id": "7bdf4525-68b7-42a1-8f40-c5f5bbbfab70", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "1571000087109", + "display": "Ultrasonography of bilateral breasts (procedure)" + } + ], + "text": "Ultrasonography of bilateral breasts (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "performedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-22T04:38:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:6433b673-1bf2-4b90-aeb2-ada4c8acd7fe", + "resource": { + "resourceType": "Procedure", + "id": "6433b673-1bf2-4b90-aeb2-ada4c8acd7fe", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "122548005", + "display": "Biopsy of breast (procedure)" + } + ], + "text": "Biopsy of breast (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "performedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-22T05:12:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7b29f70c-405e-4624-b4a3-fc3d8fdb243c", + "resource": { + "resourceType": "Procedure", + "id": "7b29f70c-405e-4624-b4a3-fc3d8fdb243c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "434363004", + "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "performedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a097d93f-d092-4a6e-957d-7ea09fb4c5c6", + "resource": { + "resourceType": "Procedure", + "id": "a097d93f-d092-4a6e-957d-7ea09fb4c5c6", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "433114000", + "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + }, + "performedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-22T06:15:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:404399f8-1e31-48a9-acf4-75803f40e1ac", + "resource": { + "resourceType": "Claim", + "id": "404399f8-1e31-48a9-acf4-75803f40e1ac", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "created": "2013-11-23T04:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:2d9bcd92-efb2-4ec2-8e94-2960f2118652" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:7bdf4525-68b7-42a1-8f40-c5f5bbbfab70" + } + }, + { + "sequence": 3, + "procedureReference": { + "reference": "urn:uuid:6433b673-1bf2-4b90-aeb2-ada4c8acd7fe" + } + }, + { + "sequence": 4, + "procedureReference": { + "reference": "urn:uuid:7b29f70c-405e-4624-b4a3-fc3d8fdb243c" + } + }, + { + "sequence": 5, + "procedureReference": { + "reference": "urn:uuid:a097d93f-d092-4a6e-957d-7ea09fb4c5c6" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "86013001", + "display": "Periodic reevaluation and management of healthy individual (procedure)" + } + ], + "text": "Periodic reevaluation and management of healthy individual (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "24623002", + "display": "Screening mammography (procedure)" + } + ], + "text": "Screening mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "1571000087109", + "display": "Ultrasonography of bilateral breasts (procedure)" + } + ], + "text": "Ultrasonography of bilateral breasts (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "122548005", + "display": "Biopsy of breast (procedure)" + } + ], + "text": "Biopsy of breast (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + }, + { + "sequence": 5, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ], + "text": "Malignant neoplasm of breast (disorder)" + } + }, + { + "sequence": 6, + "procedureSequence": [ + 4 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "434363004", + "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + }, + { + "sequence": 7, + "procedureSequence": [ + 5 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "433114000", + "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8a5c7307-4cab-44e9-aff8-2650b226dd50", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8a5c7307-4cab-44e9-aff8-2650b226dd50", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "404399f8-1e31-48a9-acf4-75803f40e1ac" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-11-23T04:15:51-08:00", + "end": "2014-11-23T04:15:51-08:00" + }, + "created": "2013-11-23T04:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:404399f8-1e31-48a9-acf4-75803f40e1ac" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "86013001", + "display": "Periodic reevaluation and management of healthy individual (procedure)" + } + ], + "text": "Periodic reevaluation and management of healthy individual (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "24623002", + "display": "Screening mammography (procedure)" + } + ], + "text": "Screening mammography (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "1571000087109", + "display": "Ultrasonography of bilateral breasts (procedure)" + } + ], + "text": "Ultrasonography of bilateral breasts (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "122548005", + "display": "Biopsy of breast (procedure)" + } + ], + "text": "Biopsy of breast (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ], + "text": "Malignant neoplasm of breast (disorder)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 6, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "434363004", + "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 7, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "433114000", + "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + } + ], + "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-22T04:15:51-08:00", + "end": "2013-11-23T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 2066.6, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50", + "resource": { + "resourceType": "Encounter", + "id": "dada8fdb-4fee-4e34-8851-6bede0d3ce50", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223484005", + "display": "Discussion about treatment (procedure)" + } + ], + "text": "Discussion about treatment (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-11-24T04:15:51-08:00", + "end": "2013-11-24T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-11-24T04:15:51-08:00", + "end": "2013-11-24T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7a4b7a71-215d-4d12-9471-cac178e23dcb", + "resource": { + "resourceType": "Claim", + "id": "7a4b7a71-215d-4d12-9471-cac178e23dcb", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-11-24T04:15:51-08:00", + "end": "2013-11-24T04:30:51-08:00" + }, + "created": "2013-11-24T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223484005", + "display": "Discussion about treatment (procedure)" + } + ], + "text": "Discussion about treatment (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0f9d824e-aed8-49e9-9fa9-bc10333bcd73", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0f9d824e-aed8-49e9-9fa9-bc10333bcd73", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7a4b7a71-215d-4d12-9471-cac178e23dcb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-11-24T04:30:51-08:00", + "end": "2014-11-24T04:30:51-08:00" + }, + "created": "2013-11-24T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7a4b7a71-215d-4d12-9471-cac178e23dcb" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223484005", + "display": "Discussion about treatment (procedure)" + } + ], + "text": "Discussion about treatment (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-24T04:15:51-08:00", + "end": "2013-11-24T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5", + "resource": { + "resourceType": "Encounter", + "id": "b542b686-f1d2-4373-a1bf-8e1b4b36d7a5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-27T06:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-27T06:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:553a9230-0e12-44c4-96be-8a1357afe96d", + "resource": { + "resourceType": "Procedure", + "id": "553a9230-0e12-44c4-96be-8a1357afe96d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "69031006", + "display": "Excision of breast tissue (procedure)" + } + ], + "text": "Excision of breast tissue (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" + }, + "performedPeriod": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-26T06:15:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:d05bf801-8a4f-487a-9a28-2672ee2973a1", + "resource": { + "resourceType": "Claim", + "id": "d05bf801-8a4f-487a-9a28-2672ee2973a1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-27T06:15:51-08:00" + }, + "created": "2013-11-27T06:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:553a9230-0e12-44c4-96be-8a1357afe96d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "69031006", + "display": "Excision of breast tissue (procedure)" + } + ], + "text": "Excision of breast tissue (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bd123d23-56e2-4825-ab16-6734a0d642cd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "bd123d23-56e2-4825-ab16-6734a0d642cd", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d05bf801-8a4f-487a-9a28-2672ee2973a1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-11-27T06:15:51-08:00", + "end": "2014-11-27T06:15:51-08:00" + }, + "created": "2013-11-27T06:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:d05bf801-8a4f-487a-9a28-2672ee2973a1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-27T06:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "69031006", + "display": "Excision of breast tissue (procedure)" + } + ], + "text": "Excision of breast tissue (procedure)" + }, + "servicedPeriod": { + "start": "2013-11-26T04:15:51-08:00", + "end": "2013-11-27T06:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71", + "resource": { + "resourceType": "Encounter", + "id": "228f3cbf-e4fe-4f71-a4e8-0c494fd44d71", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:331cf1fe-c9ea-4cf7-98b3-2ac10deea32c", + "resource": { + "resourceType": "Procedure", + "id": "331cf1fe-c9ea-4cf7-98b3-2ac10deea32c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" + }, + "performedPeriod": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-05T04:15:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b8e9bf83-736c-4a49-802b-f2bf65dc5fdb", + "resource": { + "resourceType": "MedicationRequest", + "id": "b8e9bf83-736c-4a49-802b-f2bf65dc5fdb", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" + }, + "authoredOn": "2013-12-05T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:27c611a5-0a2c-4ff4-8fdc-c014dce6e23e", + "resource": { + "resourceType": "Claim", + "id": "27c611a5-0a2c-4ff4-8fdc-c014dce6e23e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "created": "2013-12-06T04:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:b8e9bf83-736c-4a49-802b-f2bf65dc5fdb" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" + } + ] + } + ], + "total": { + "value": 6470.39, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:040ddc05-c3c4-4a78-9e59-7dd25634607e", + "resource": { + "resourceType": "Claim", + "id": "040ddc05-c3c4-4a78-9e59-7dd25634607e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "created": "2013-12-06T04:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:331cf1fe-c9ea-4cf7-98b3-2ac10deea32c" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2650dbb5-1e6b-43d1-bba2-5f03860a177f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2650dbb5-1e6b-43d1-bba2-5f03860a177f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "040ddc05-c3c4-4a78-9e59-7dd25634607e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-12-06T04:15:51-08:00", + "end": "2014-12-06T04:15:51-08:00" + }, + "created": "2013-12-06T04:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:040ddc05-c3c4-4a78-9e59-7dd25634607e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-12-05T04:15:51-08:00", + "end": "2013-12-06T04:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb", + "resource": { + "resourceType": "Encounter", + "id": "16c7951f-92b6-44ac-b75a-1323d58319fb", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c332b370-eb9d-4ee3-9a6f-060c6e888330", + "resource": { + "resourceType": "Procedure", + "id": "c332b370-eb9d-4ee3-9a6f-060c6e888330", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" + }, + "performedPeriod": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-27T05:15:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:55c2af1b-0077-4dcd-ac3c-a4446e76ff88", + "resource": { + "resourceType": "MedicationRequest", + "id": "55c2af1b-0077-4dcd-ac3c-a4446e76ff88", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" + }, + "authoredOn": "2013-12-27T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:b433dbc1-938d-46d2-9934-75fa59e23583", + "resource": { + "resourceType": "Claim", + "id": "b433dbc1-938d-46d2-9934-75fa59e23583", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "created": "2013-12-28T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:55c2af1b-0077-4dcd-ac3c-a4446e76ff88" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" + } + ] + } + ], + "total": { + "value": 6389.43, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fd2c5e5b-d3a1-47c9-84e9-0d957881497f", + "resource": { + "resourceType": "Claim", + "id": "fd2c5e5b-d3a1-47c9-84e9-0d957881497f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "created": "2013-12-28T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:c332b370-eb9d-4ee3-9a6f-060c6e888330" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:65dc99d0-29bc-4cbb-b24b-b0da621c633a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "65dc99d0-29bc-4cbb-b24b-b0da621c633a", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fd2c5e5b-d3a1-47c9-84e9-0d957881497f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2013-12-28T05:15:51-08:00", + "end": "2014-12-28T05:15:51-08:00" + }, + "created": "2013-12-28T05:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:fd2c5e5b-d3a1-47c9-84e9-0d957881497f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2013-12-27T04:15:51-08:00", + "end": "2013-12-28T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c", + "resource": { + "resourceType": "Encounter", + "id": "8c4f2b96-5ca1-4014-8945-b77541808e6c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d7f60c87-8fff-4f41-b97b-bb5477f09efe", + "resource": { + "resourceType": "Procedure", + "id": "d7f60c87-8fff-4f41-b97b-bb5477f09efe", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" + }, + "performedPeriod": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-16T05:15:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:fcd92e33-245f-48d9-a7a3-d8b61f4d2a38", + "resource": { + "resourceType": "MedicationRequest", + "id": "fcd92e33-245f-48d9-a7a3-d8b61f4d2a38", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" + }, + "authoredOn": "2014-01-16T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:3767dc7d-1c1f-4cf8-982e-59c4c9d79113", + "resource": { + "resourceType": "Claim", + "id": "3767dc7d-1c1f-4cf8-982e-59c4c9d79113", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "created": "2014-01-17T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:fcd92e33-245f-48d9-a7a3-d8b61f4d2a38" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" + } + ] + } + ], + "total": { + "value": 6515.37, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:96a9eeb6-9b95-4f26-9166-851a5262fcd1", + "resource": { + "resourceType": "Claim", + "id": "96a9eeb6-9b95-4f26-9166-851a5262fcd1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "created": "2014-01-17T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:d7f60c87-8fff-4f41-b97b-bb5477f09efe" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d87eac2a-83ba-451f-86fb-b522336977cb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d87eac2a-83ba-451f-86fb-b522336977cb", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "96a9eeb6-9b95-4f26-9166-851a5262fcd1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-01-17T05:15:51-08:00", + "end": "2015-01-17T05:15:51-08:00" + }, + "created": "2014-01-17T05:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:96a9eeb6-9b95-4f26-9166-851a5262fcd1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-01-16T04:15:51-08:00", + "end": "2014-01-17T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a", + "resource": { + "resourceType": "Encounter", + "id": "b7493584-25de-4839-894c-dec5e46a500a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5673bd55-4554-4174-ad19-191760da1d52", + "resource": { + "resourceType": "Procedure", + "id": "5673bd55-4554-4174-ad19-191760da1d52", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" + }, + "performedPeriod": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-07T05:15:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:2d84e83e-18a3-4636-9c3c-0442cbf4c037", + "resource": { + "resourceType": "MedicationRequest", + "id": "2d84e83e-18a3-4636-9c3c-0442cbf4c037", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" + }, + "authoredOn": "2014-02-07T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:550a3cc6-ff13-481b-a696-10ed812d9de4", + "resource": { + "resourceType": "Claim", + "id": "550a3cc6-ff13-481b-a696-10ed812d9de4", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "created": "2014-02-08T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:2d84e83e-18a3-4636-9c3c-0442cbf4c037" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" + } + ] + } + ], + "total": { + "value": 6289.71, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:63563296-992c-40b8-bee8-fc4908e779d5", + "resource": { + "resourceType": "Claim", + "id": "63563296-992c-40b8-bee8-fc4908e779d5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "created": "2014-02-08T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:5673bd55-4554-4174-ad19-191760da1d52" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5903532d-1c7d-4c74-b547-2fdd0911e42d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5903532d-1c7d-4c74-b547-2fdd0911e42d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "63563296-992c-40b8-bee8-fc4908e779d5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-02-08T05:15:51-08:00", + "end": "2015-02-08T05:15:51-08:00" + }, + "created": "2014-02-08T05:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:63563296-992c-40b8-bee8-fc4908e779d5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-02-07T04:15:51-08:00", + "end": "2014-02-08T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8", + "resource": { + "resourceType": "Encounter", + "id": "f05e59fc-b1f2-4ef4-b29f-30c0860d86b8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:eac54ae8-28e3-4372-9343-408b7e9eb936", + "resource": { + "resourceType": "Procedure", + "id": "eac54ae8-28e3-4372-9343-408b7e9eb936", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" + }, + "performedPeriod": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-02-28T05:15:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:6a3de65f-6f05-4166-8b8a-34657d72794d", + "resource": { + "resourceType": "MedicationRequest", + "id": "6a3de65f-6f05-4166-8b8a-34657d72794d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" + }, + "authoredOn": "2014-02-28T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:950faa81-e44f-481e-878a-be9454913c9f", + "resource": { + "resourceType": "Claim", + "id": "950faa81-e44f-481e-878a-be9454913c9f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "created": "2014-03-01T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:6a3de65f-6f05-4166-8b8a-34657d72794d" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" + } + ] + } + ], + "total": { + "value": 6623.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2633abc9-222b-4a73-9f00-599bf2a5d7bd", + "resource": { + "resourceType": "Claim", + "id": "2633abc9-222b-4a73-9f00-599bf2a5d7bd", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "created": "2014-03-01T05:15:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:eac54ae8-28e3-4372-9343-408b7e9eb936" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1bccd126-9798-41da-9f59-0f0e58db4ef8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1bccd126-9798-41da-9f59-0f0e58db4ef8", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2633abc9-222b-4a73-9f00-599bf2a5d7bd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-03-01T05:15:51-08:00", + "end": "2015-03-01T05:15:51-08:00" + }, + "created": "2014-03-01T05:15:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2633abc9-222b-4a73-9f00-599bf2a5d7bd" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-02-28T04:15:51-08:00", + "end": "2014-03-01T05:15:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d", + "resource": { + "resourceType": "Encounter", + "id": "51e879d8-de42-4413-88c1-4e24fae5803d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e69dbe01-13f8-4dd9-8f1b-54488e359385", + "resource": { + "resourceType": "Procedure", + "id": "e69dbe01-13f8-4dd9-8f1b-54488e359385", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" + }, + "performedPeriod": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-21T06:15:51-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:11df9d03-997a-4999-ac9b-915c7533be1f", + "resource": { + "resourceType": "MedicationRequest", + "id": "11df9d03-997a-4999-ac9b-915c7533be1f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" + }, + "authoredOn": "2014-03-21T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5e345c16-e322-47bc-9b7a-a5c746a02d23", + "resource": { + "resourceType": "Claim", + "id": "5e345c16-e322-47bc-9b7a-a5c746a02d23", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "created": "2014-03-22T06:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:11df9d03-997a-4999-ac9b-915c7533be1f" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" + } + ] + } + ], + "total": { + "value": 6500.38, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:64da860e-8923-438a-bdc5-d18c4fc7c17a", + "resource": { + "resourceType": "Claim", + "id": "64da860e-8923-438a-bdc5-d18c4fc7c17a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "created": "2014-03-22T06:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e69dbe01-13f8-4dd9-8f1b-54488e359385" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e8d72d04-3a14-4dd0-96c4-34d6f5c5c05d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e8d72d04-3a14-4dd0-96c4-34d6f5c5c05d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "64da860e-8923-438a-bdc5-d18c4fc7c17a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-03-22T06:15:51-07:00", + "end": "2015-03-22T06:15:51-07:00" + }, + "created": "2014-03-22T06:15:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:64da860e-8923-438a-bdc5-d18c4fc7c17a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-03-21T05:15:51-07:00", + "end": "2014-03-22T06:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5", + "resource": { + "resourceType": "Encounter", + "id": "328a23f5-ab37-4cd5-aa29-092180c8bfc5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3027d537-9a35-4aee-8f0f-0d08352a222d", + "resource": { + "resourceType": "Procedure", + "id": "3027d537-9a35-4aee-8f0f-0d08352a222d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" + }, + "performedPeriod": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-10T06:15:51-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:014e5434-d1c6-4251-a6f4-06635b85f3d2", + "resource": { + "resourceType": "MedicationRequest", + "id": "014e5434-d1c6-4251-a6f4-06635b85f3d2", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" + }, + "authoredOn": "2014-04-10T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:2a3893a6-bdbb-476b-8b5b-1acde08b71f7", + "resource": { + "resourceType": "Claim", + "id": "2a3893a6-bdbb-476b-8b5b-1acde08b71f7", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "created": "2014-04-11T06:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:014e5434-d1c6-4251-a6f4-06635b85f3d2" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" + } + ] + } + ], + "total": { + "value": 6432.3, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2a7128b6-73ed-443f-82be-2ada7529b87a", + "resource": { + "resourceType": "Claim", + "id": "2a7128b6-73ed-443f-82be-2ada7529b87a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "created": "2014-04-11T06:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:3027d537-9a35-4aee-8f0f-0d08352a222d" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:10f99655-874d-4dae-8f54-0187a578cba5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "10f99655-874d-4dae-8f54-0187a578cba5", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2a7128b6-73ed-443f-82be-2ada7529b87a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-04-11T06:15:51-07:00", + "end": "2015-04-11T06:15:51-07:00" + }, + "created": "2014-04-11T06:15:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:2a7128b6-73ed-443f-82be-2ada7529b87a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-04-10T05:15:51-07:00", + "end": "2014-04-11T06:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54", + "resource": { + "resourceType": "Encounter", + "id": "e6d36055-a5bc-434f-ba14-70b08f1ddc54", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6fbd1be0-04c7-420b-ba56-a213fab757ad", + "resource": { + "resourceType": "Procedure", + "id": "6fbd1be0-04c7-420b-ba56-a213fab757ad", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" + }, + "performedPeriod": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-01T05:15:51-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e9fa9475-38ce-4e90-a55d-afc3c0f1bfab", + "resource": { + "resourceType": "MedicationRequest", + "id": "e9fa9475-38ce-4e90-a55d-afc3c0f1bfab", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "583214", + "display": "Paclitaxel 100 MG Injection" + } + ], + "text": "Paclitaxel 100 MG Injection" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" + }, + "authoredOn": "2014-05-01T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:781787e7-9169-4fc9-b955-a2d45b874b08", + "resource": { + "resourceType": "Claim", + "id": "781787e7-9169-4fc9-b955-a2d45b874b08", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "created": "2014-05-02T05:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:e9fa9475-38ce-4e90-a55d-afc3c0f1bfab" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" + } + ] + } + ], + "total": { + "value": 6906.05, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3764e02f-7125-4610-a1fe-3d255b6df4b9", + "resource": { + "resourceType": "Claim", + "id": "3764e02f-7125-4610-a1fe-3d255b6df4b9", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "created": "2014-05-02T05:15:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:6fbd1be0-04c7-420b-ba56-a213fab757ad" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:95e4e1dd-91d1-405d-afa4-1c4cadbda3b0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "95e4e1dd-91d1-405d-afa4-1c4cadbda3b0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3764e02f-7125-4610-a1fe-3d255b6df4b9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-05-02T05:15:51-07:00", + "end": "2015-05-02T05:15:51-07:00" + }, + "created": "2014-05-02T05:15:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3764e02f-7125-4610-a1fe-3d255b6df4b9" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "367336001", + "display": "Chemotherapy (procedure)" + } + ], + "text": "Chemotherapy (procedure)" + }, + "servicedPeriod": { + "start": "2014-05-01T05:15:51-07:00", + "end": "2014-05-02T05:15:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0", + "resource": { + "resourceType": "Encounter", + "id": "e0f6735b-abb1-4c75-8bd9-64ff8a5851a0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-05-12T05:15:51-07:00", + "end": "2014-05-12T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-05-12T05:15:51-07:00", + "end": "2014-05-12T05:30:51-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:367e3a46-b1ea-4e7e-90bc-bd74398d836e", + "resource": { + "resourceType": "Observation", + "id": "367e3a46-b1ea-4e7e-90bc-bd74398d836e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "procedure", + "display": "procedure" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59557-9", + "display": "Treatment status Cancer" + } + ], + "text": "Treatment status Cancer" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + }, + "effectiveDateTime": "2014-05-12T05:15:51-07:00", + "issued": "2014-05-12T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "445528004", + "display": "Treatment changed (situation)" + } + ], + "text": "Treatment changed (situation)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b30ae9d-2aa8-4be7-99b9-2bad98a48e5d", + "resource": { + "resourceType": "Observation", + "id": "1b30ae9d-2aa8-4be7-99b9-2bad98a48e5d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "therapy", + "display": "therapy" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "88040-1", + "display": "Response to cancer treatment" + } + ], + "text": "Response to cancer treatment" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + }, + "effectiveDateTime": "2014-05-12T05:15:51-07:00", + "issued": "2014-05-12T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385633008", + "display": "Improving (qualifier value)" + } + ], + "text": "Improving (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c55d5924-a63c-431f-8f06-107404ce70cb", + "resource": { + "resourceType": "MedicationRequest", + "id": "c55d5924-a63c-431f-8f06-107404ce70cb", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "199224", + "display": "anastrozole 1 MG Oral Tablet" + } + ], + "text": "anastrozole 1 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + }, + "authoredOn": "2014-05-12T05:15:51-07:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:02d62831-7398-4845-aa65-880e0ca6eb20", + "resource": { + "resourceType": "Claim", + "id": "02d62831-7398-4845-aa65-880e0ca6eb20", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-05-12T05:15:51-07:00", + "end": "2014-05-12T05:30:51-07:00" + }, + "created": "2014-05-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:c55d5924-a63c-431f-8f06-107404ce70cb" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c018c44a-eca8-4355-aaf2-8fe014d18b43", + "resource": { + "resourceType": "Claim", + "id": "c018c44a-eca8-4355-aaf2-8fe014d18b43", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-05-12T05:15:51-07:00", + "end": "2014-05-12T05:30:51-07:00" + }, + "created": "2014-05-12T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "encounter": [ + { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:006bd392-f7b4-432c-a080-f79e2b50780f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "006bd392-f7b4-432c-a080-f79e2b50780f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c018c44a-eca8-4355-aaf2-8fe014d18b43" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-05-12T05:30:51-07:00", + "end": "2015-05-12T05:30:51-07:00" + }, + "created": "2014-05-12T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:c018c44a-eca8-4355-aaf2-8fe014d18b43" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + }, + "servicedPeriod": { + "start": "2014-05-12T05:15:51-07:00", + "end": "2014-05-12T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8", + "resource": { + "resourceType": "Encounter", + "id": "a9b8805c-69da-45a0-bfb4-e7a0a676b4a8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-08-18T05:15:51-07:00", + "end": "2014-08-18T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-08-18T05:15:51-07:00", + "end": "2014-08-18T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:07c40acd-88fd-4224-a72e-ba0263e587d5", + "resource": { + "resourceType": "Claim", + "id": "07c40acd-88fd-4224-a72e-ba0263e587d5", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-08-18T05:15:51-07:00", + "end": "2014-08-18T05:30:51-07:00" + }, + "created": "2014-08-18T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6f2abedf-483a-411d-995f-463a96654c1e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6f2abedf-483a-411d-995f-463a96654c1e", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "07c40acd-88fd-4224-a72e-ba0263e587d5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-08-18T05:30:51-07:00", + "end": "2015-08-18T05:30:51-07:00" + }, + "created": "2014-08-18T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:07c40acd-88fd-4224-a72e-ba0263e587d5" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2014-08-18T05:15:51-07:00", + "end": "2014-08-18T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7", + "resource": { + "resourceType": "Encounter", + "id": "53766e80-9208-4520-9598-d6d152bd04b7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:45e8cad5-7d34-46c2-9ea9-c87f5fbe4273", + "resource": { + "resourceType": "Observation", + "id": "45e8cad5-7d34-46c2-9ea9-c87f5fbe4273", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:163e8daa-97ba-4518-bd1d-c002de45ab7c", + "resource": { + "resourceType": "Observation", + "id": "163e8daa-97ba-4518-bd1d-c002de45ab7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1b6077cf-2555-47a7-9266-db6539d86e45", + "resource": { + "resourceType": "Observation", + "id": "1b6077cf-2555-47a7-9266-db6539d86e45", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eac88c1e-ab1c-47f6-8c12-e75925934134", + "resource": { + "resourceType": "Observation", + "id": "eac88c1e-ab1c-47f6-8c12-e75925934134", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f4bc8640-a954-41a9-a33e-3840537c2978", + "resource": { + "resourceType": "Observation", + "id": "f4bc8640-a954-41a9-a33e-3840537c2978", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 125, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4dc17dee-b490-48fc-a062-72428add11a9", + "resource": { + "resourceType": "Observation", + "id": "4dc17dee-b490-48fc-a062-72428add11a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 94, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25469286-28f1-4a51-bc9a-379d021a9592", + "resource": { + "resourceType": "Observation", + "id": "25469286-28f1-4a51-bc9a-379d021a9592", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:16b81c2e-49e2-4718-a03f-b0ea902fde5a", + "resource": { + "resourceType": "Observation", + "id": "16b81c2e-49e2-4718-a03f-b0ea902fde5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 69.71, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0faca098-17c3-4cc1-9f4f-e53a482a3de7", + "resource": { + "resourceType": "Observation", + "id": "0faca098-17c3-4cc1-9f4f-e53a482a3de7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.89, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c6472ec-abc2-4c56-801c-5b007a1c0d02", + "resource": { + "resourceType": "Observation", + "id": "4c6472ec-abc2-4c56-801c-5b007a1c0d02", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.7, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d9e50bdc-a8fb-4305-9690-b361a8f310b6", + "resource": { + "resourceType": "Observation", + "id": "d9e50bdc-a8fb-4305-9690-b361a8f310b6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 8.85, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71052ac5-b9fe-45e9-95e8-e3075e573983", + "resource": { + "resourceType": "Observation", + "id": "71052ac5-b9fe-45e9-95e8-e3075e573983", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 143.17, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d9c2e28-572b-4d11-8073-178e716da083", + "resource": { + "resourceType": "Observation", + "id": "1d9c2e28-572b-4d11-8073-178e716da083", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 3.9, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", + "resource": { + "resourceType": "Observation", + "id": "02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 106.66, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69090f27-b9d4-4969-aec4-fcad583c19c5", + "resource": { + "resourceType": "Observation", + "id": "69090f27-b9d4-4969-aec4-fcad583c19c5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 26.96, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6aeb4914-9cba-4c32-ab9c-03423548dbce", + "resource": { + "resourceType": "Observation", + "id": "6aeb4914-9cba-4c32-ab9c-03423548dbce", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cb12980-f522-4eb4-b071-f6fe06afed50", + "resource": { + "resourceType": "Observation", + "id": "0cb12980-f522-4eb4-b071-f6fe06afed50", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.12, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:05574e52-2809-4e95-8df2-a700f6856f11", + "resource": { + "resourceType": "MedicationRequest", + "id": "05574e52-2809-4e95-8df2-a700f6856f11", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "authoredOn": "2014-11-10T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1d5bafe1-5c29-4036-9a5a-d8b76df7cfbf", + "resource": { + "resourceType": "Claim", + "id": "1d5bafe1-5c29-4036-9a5a-d8b76df7cfbf", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "created": "2014-11-10T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:05574e52-2809-4e95-8df2-a700f6856f11" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:06166efc-fab7-4872-8e46-b0f4bf1f5de6", + "resource": { + "resourceType": "Immunization", + "id": "06166efc-fab7-4872-8e46-b0f4bf1f5de6", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "occurrenceDateTime": "2014-11-10T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4442da8a-40bd-47b7-b640-48577dc5942c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4442da8a-40bd-47b7-b640-48577dc5942c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + }, + "effectiveDateTime": "2014-11-10T04:15:51-08:00", + "issued": "2014-11-10T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:16b81c2e-49e2-4718-a03f-b0ea902fde5a", + "display": "Glucose" + }, + { + "reference": "urn:uuid:0faca098-17c3-4cc1-9f4f-e53a482a3de7", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:4c6472ec-abc2-4c56-801c-5b007a1c0d02", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:d9e50bdc-a8fb-4305-9690-b361a8f310b6", + "display": "Calcium" + }, + { + "reference": "urn:uuid:71052ac5-b9fe-45e9-95e8-e3075e573983", + "display": "Sodium" + }, + { + "reference": "urn:uuid:1d9c2e28-572b-4d11-8073-178e716da083", + "display": "Potassium" + }, + { + "reference": "urn:uuid:02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", + "display": "Chloride" + }, + { + "reference": "urn:uuid:69090f27-b9d4-4969-aec4-fcad583c19c5", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:be513f2a-5fe4-427b-8036-450e0b806f94", + "resource": { + "resourceType": "Claim", + "id": "be513f2a-5fe4-427b-8036-450e0b806f94", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "created": "2014-11-10T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:06166efc-fab7-4872-8e46-b0f4bf1f5de6" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e9cc25c0-dad7-476c-a21a-9dae3b37efa3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e9cc25c0-dad7-476c-a21a-9dae3b37efa3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "be513f2a-5fe4-427b-8036-450e0b806f94" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-11-10T04:30:51-08:00", + "end": "2015-11-10T04:30:51-08:00" + }, + "created": "2014-11-10T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:be513f2a-5fe4-427b-8036-450e0b806f94" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2014-11-10T04:15:51-08:00", + "end": "2014-11-10T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f", + "resource": { + "resourceType": "Encounter", + "id": "9a33c653-4d89-4e7f-b827-3ea7774e064f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2014-12-08T04:15:51-08:00", + "end": "2014-12-08T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2014-12-08T04:15:51-08:00", + "end": "2014-12-08T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9d28d844-2bc4-4680-ae36-3d1b91cfd059", + "resource": { + "resourceType": "Claim", + "id": "9d28d844-2bc4-4680-ae36-3d1b91cfd059", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2014-12-08T04:15:51-08:00", + "end": "2014-12-08T04:30:51-08:00" + }, + "created": "2014-12-08T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cd37a371-76f3-410c-8876-c5e49f432762", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cd37a371-76f3-410c-8876-c5e49f432762", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9d28d844-2bc4-4680-ae36-3d1b91cfd059" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2014-12-08T04:30:51-08:00", + "end": "2015-12-08T04:30:51-08:00" + }, + "created": "2014-12-08T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9d28d844-2bc4-4680-ae36-3d1b91cfd059" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2014-12-08T04:15:51-08:00", + "end": "2014-12-08T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83", + "resource": { + "resourceType": "Encounter", + "id": "cc631c4c-782d-42bf-9bd4-ae8b9db2bc83", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:239f47b9-8621-415e-a4a1-d6df31f681ea", + "resource": { + "resourceType": "Claim", + "id": "239f47b9-8621-415e-a4a1-d6df31f681ea", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:30:51-07:00" + }, + "created": "2015-03-23T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0502bcb2-171b-4067-b713-cc6b181ddaf2", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0502bcb2-171b-4067-b713-cc6b181ddaf2", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "239f47b9-8621-415e-a4a1-d6df31f681ea" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-03-23T05:30:51-07:00", + "end": "2016-03-23T05:30:51-07:00" + }, + "created": "2015-03-23T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:239f47b9-8621-415e-a4a1-d6df31f681ea" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f", + "resource": { + "resourceType": "Encounter", + "id": "db4e5e28-8de2-401b-8c4b-9e678d4ed30f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-24T05:38:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-24T05:38:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945", + "resource": { + "resourceType": "Procedure", + "id": "4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" + }, + "performedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:38:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7023c212-9ed4-4522-965a-127120a89fe3", + "resource": { + "resourceType": "Claim", + "id": "7023c212-9ed4-4522-965a-127120a89fe3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-24T05:38:51-07:00" + }, + "created": "2015-03-24T05:38:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:18ca771b-37cd-411b-8bd9-fb5e291d31cc", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "18ca771b-37cd-411b-8bd9-fb5e291d31cc", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7023c212-9ed4-4522-965a-127120a89fe3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-03-24T05:38:51-07:00", + "end": "2016-03-24T05:38:51-07:00" + }, + "created": "2015-03-24T05:38:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7023c212-9ed4-4522-965a-127120a89fe3" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-24T05:38:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-24T05:38:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9", + "resource": { + "resourceType": "Encounter", + "id": "c62a7444-0148-4e6f-b341-f775b22bbfb9", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:57:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:57:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:95d309d0-1db8-4746-bc6a-01332f2d132b", + "resource": { + "resourceType": "Observation", + "id": "95d309d0-1db8-4746-bc6a-01332f2d132b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "therapy", + "display": "therapy" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "88040-1", + "display": "Response to cancer treatment" + } + ], + "text": "Response to cancer treatment" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" + }, + "effectiveDateTime": "2015-03-23T05:15:51-07:00", + "issued": "2015-03-23T05:15:51.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385633008", + "display": "Improving (qualifier value)" + } + ], + "text": "Improving (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3", + "resource": { + "resourceType": "Procedure", + "id": "ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" + }, + "performedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:42:51-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:cb1e28e0-c4a1-4843-8cc0-68caca58f6e2", + "resource": { + "resourceType": "Claim", + "id": "cb1e28e0-c4a1-4843-8cc0-68caca58f6e2", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:57:51-07:00" + }, + "created": "2015-03-23T05:57:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 15112.10, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:430b4506-96ad-4f5a-b375-d244d896ad25", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "430b4506-96ad-4f5a-b375-d244d896ad25", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cb1e28e0-c4a1-4843-8cc0-68caca58f6e2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-03-23T05:57:51-07:00", + "end": "2016-03-23T05:57:51-07:00" + }, + "created": "2015-03-23T05:57:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:cb1e28e0-c4a1-4843-8cc0-68caca58f6e2" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:57:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2015-03-23T05:15:51-07:00", + "end": "2015-03-23T05:57:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 15112.10, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 3022.42, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 12089.68, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 15112.10, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 15112.10, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 12089.68, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439", + "resource": { + "resourceType": "Encounter", + "id": "bf22e432-063f-4b85-b8fd-7774f7294439", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-07-13T05:15:51-07:00", + "end": "2015-07-13T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-07-13T05:15:51-07:00", + "end": "2015-07-13T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1", + "resource": { + "resourceType": "Claim", + "id": "aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-07-13T05:15:51-07:00", + "end": "2015-07-13T05:30:51-07:00" + }, + "created": "2015-07-13T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b9b6b7fb-6d19-4449-ae8f-43b4a6d0a2ac", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b9b6b7fb-6d19-4449-ae8f-43b4a6d0a2ac", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-07-13T05:30:51-07:00", + "end": "2016-07-13T05:30:51-07:00" + }, + "created": "2015-07-13T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2015-07-13T05:15:51-07:00", + "end": "2015-07-13T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b", + "resource": { + "resourceType": "Encounter", + "id": "88775039-b70f-4941-adab-6b1b01cf046b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-07-23T05:15:51-07:00", + "end": "2015-07-23T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-07-23T05:15:51-07:00", + "end": "2015-07-23T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "109838007", + "display": "Overlapping malignant neoplasm of colon" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ca8c84f5-d23b-4824-8746-f8a290a7c05e", + "resource": { + "resourceType": "Claim", + "id": "ca8c84f5-d23b-4824-8746-f8a290a7c05e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-07-23T05:15:51-07:00", + "end": "2015-07-23T05:30:51-07:00" + }, + "created": "2015-07-23T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9259701d-d0d5-405a-891a-a1199788a08e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9259701d-d0d5-405a-891a-a1199788a08e", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ca8c84f5-d23b-4824-8746-f8a290a7c05e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-07-23T05:30:51-07:00", + "end": "2016-07-23T05:30:51-07:00" + }, + "created": "2015-07-23T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ca8c84f5-d23b-4824-8746-f8a290a7c05e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2015-07-23T05:15:51-07:00", + "end": "2015-07-23T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1", + "resource": { + "resourceType": "Encounter", + "id": "818ecd47-9f63-4ead-8937-dae05f3079c1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-10-19T05:15:51-07:00", + "end": "2015-10-19T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2015-10-19T05:15:51-07:00", + "end": "2015-10-19T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9b72a763-0c0b-409e-9ca6-571ae5c6c15e", + "resource": { + "resourceType": "Claim", + "id": "9b72a763-0c0b-409e-9ca6-571ae5c6c15e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-10-19T05:15:51-07:00", + "end": "2015-10-19T05:30:51-07:00" + }, + "created": "2015-10-19T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4cdae89d-dc39-4a7d-9768-1918080d6f72", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4cdae89d-dc39-4a7d-9768-1918080d6f72", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9b72a763-0c0b-409e-9ca6-571ae5c6c15e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-10-19T05:30:51-07:00", + "end": "2016-10-19T05:30:51-07:00" + }, + "created": "2015-10-19T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9b72a763-0c0b-409e-9ca6-571ae5c6c15e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2015-10-19T05:15:51-07:00", + "end": "2015-10-19T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77", + "resource": { + "resourceType": "Encounter", + "id": "2583e8ac-f7b4-41b9-b9cb-e80a38d44a77", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:56a02f6a-4d0f-414c-b249-de948b9e8f20", + "resource": { + "resourceType": "Observation", + "id": "56a02f6a-4d0f-414c-b249-de948b9e8f20", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:050720d7-e226-4920-b4cf-a33561f211d4", + "resource": { + "resourceType": "Observation", + "id": "050720d7-e226-4920-b4cf-a33561f211d4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:178db7e9-e324-4932-bb75-bdc4df409b5a", + "resource": { + "resourceType": "Observation", + "id": "178db7e9-e324-4932-bb75-bdc4df409b5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d9c19adb-0eba-4bec-a156-85ce1b7ea7ec", + "resource": { + "resourceType": "Observation", + "id": "d9c19adb-0eba-4bec-a156-85ce1b7ea7ec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2e451479-0f1d-42d0-8fac-bf47e66cf564", + "resource": { + "resourceType": "Observation", + "id": "2e451479-0f1d-42d0-8fac-bf47e66cf564", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 109, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d9c6cbd1-d7dc-4419-98c1-f7d43568ad1b", + "resource": { + "resourceType": "Observation", + "id": "d9c6cbd1-d7dc-4419-98c1-f7d43568ad1b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 78, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:430ce9a9-94c8-433f-8547-837f237e5bb7", + "resource": { + "resourceType": "Observation", + "id": "430ce9a9-94c8-433f-8547-837f237e5bb7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65104e37-d772-42e9-ab0a-c793d779e36f", + "resource": { + "resourceType": "Observation", + "id": "65104e37-d772-42e9-ab0a-c793d779e36f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 74.84, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d14c63db-3889-4321-a2b9-2698d9c0e258", + "resource": { + "resourceType": "Observation", + "id": "d14c63db-3889-4321-a2b9-2698d9c0e258", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 13.23, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ae8f93b9-b1c5-4564-a8dd-882f69207ec3", + "resource": { + "resourceType": "Observation", + "id": "ae8f93b9-b1c5-4564-a8dd-882f69207ec3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.69, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ca7717d-07cf-4391-bfd5-d311ff630610", + "resource": { + "resourceType": "Observation", + "id": "1ca7717d-07cf-4391-bfd5-d311ff630610", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.23, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9cab820-08d5-4167-acd2-4473c30d15e9", + "resource": { + "resourceType": "Observation", + "id": "a9cab820-08d5-4167-acd2-4473c30d15e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 138.89, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e7e10938-6cc0-4069-9f82-aa983e806b19", + "resource": { + "resourceType": "Observation", + "id": "e7e10938-6cc0-4069-9f82-aa983e806b19", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 3.76, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d43a719-33ad-47a3-bf44-ebe624c4d77e", + "resource": { + "resourceType": "Observation", + "id": "0d43a719-33ad-47a3-bf44-ebe624c4d77e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 109.42, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:397a8bdc-fe8e-47ec-91a8-26da70f09819", + "resource": { + "resourceType": "Observation", + "id": "397a8bdc-fe8e-47ec-91a8-26da70f09819", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 21.34, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1fbb14d8-653e-496f-a2fa-da7db6cd15b8", + "resource": { + "resourceType": "Observation", + "id": "1fbb14d8-653e-496f-a2fa-da7db6cd15b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28fc3a4f-8686-46ab-acda-7f4b898399ee", + "resource": { + "resourceType": "Observation", + "id": "28fc3a4f-8686-46ab-acda-7f4b898399ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "valueQuantity": { + "value": 5.96, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25869f6e-53aa-40ff-9d9b-e9d0feada885", + "resource": { + "resourceType": "MedicationRequest", + "id": "25869f6e-53aa-40ff-9d9b-e9d0feada885", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "authoredOn": "2015-11-16T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:90c5f7c5-8f99-4ae3-a947-20c1d952aadf", + "resource": { + "resourceType": "Claim", + "id": "90c5f7c5-8f99-4ae3-a947-20c1d952aadf", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "created": "2015-11-16T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:25869f6e-53aa-40ff-9d9b-e9d0feada885" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7d2e7923-1783-4589-87bb-e01d1981cb03", + "resource": { + "resourceType": "Immunization", + "id": "7d2e7923-1783-4589-87bb-e01d1981cb03", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "occurrenceDateTime": "2015-11-16T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:00f0866e-f982-4001-ae1d-58f2c0c87779", + "resource": { + "resourceType": "DiagnosticReport", + "id": "00f0866e-f982-4001-ae1d-58f2c0c87779", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + }, + "effectiveDateTime": "2015-11-16T04:15:51-08:00", + "issued": "2015-11-16T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:65104e37-d772-42e9-ab0a-c793d779e36f", + "display": "Glucose" + }, + { + "reference": "urn:uuid:d14c63db-3889-4321-a2b9-2698d9c0e258", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:ae8f93b9-b1c5-4564-a8dd-882f69207ec3", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:1ca7717d-07cf-4391-bfd5-d311ff630610", + "display": "Calcium" + }, + { + "reference": "urn:uuid:a9cab820-08d5-4167-acd2-4473c30d15e9", + "display": "Sodium" + }, + { + "reference": "urn:uuid:e7e10938-6cc0-4069-9f82-aa983e806b19", + "display": "Potassium" + }, + { + "reference": "urn:uuid:0d43a719-33ad-47a3-bf44-ebe624c4d77e", + "display": "Chloride" + }, + { + "reference": "urn:uuid:397a8bdc-fe8e-47ec-91a8-26da70f09819", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:ba9a039d-0ef0-43c1-9810-0fff2c723f35", + "resource": { + "resourceType": "Claim", + "id": "ba9a039d-0ef0-43c1-9810-0fff2c723f35", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "created": "2015-11-16T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7d2e7923-1783-4589-87bb-e01d1981cb03" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:804fbaa8-0721-48a7-ae59-b20a07fc2905", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "804fbaa8-0721-48a7-ae59-b20a07fc2905", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ba9a039d-0ef0-43c1-9810-0fff2c723f35" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2015-11-16T04:30:51-08:00", + "end": "2016-11-16T04:30:51-08:00" + }, + "created": "2015-11-16T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ba9a039d-0ef0-43c1-9810-0fff2c723f35" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2015-11-16T04:15:51-08:00", + "end": "2015-11-16T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d", + "resource": { + "resourceType": "Encounter", + "id": "aa56cec4-c00a-4454-a033-76b56580687d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:74e8aea0-2a43-4176-9e04-fd91e9613dd8", + "resource": { + "resourceType": "Claim", + "id": "74e8aea0-2a43-4176-9e04-fd91e9613dd8", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:30:51-08:00" + }, + "created": "2016-02-15T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:47c945ba-b7ba-4afa-9bf7-7f001fa90751", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "47c945ba-b7ba-4afa-9bf7-7f001fa90751", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "74e8aea0-2a43-4176-9e04-fd91e9613dd8" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-02-15T04:30:51-08:00", + "end": "2017-02-15T04:30:51-08:00" + }, + "created": "2016-02-15T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:74e8aea0-2a43-4176-9e04-fd91e9613dd8" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3", + "resource": { + "resourceType": "Encounter", + "id": "6765d1a2-4909-4a35-88b1-5f1eb690e7b3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-16T04:34:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-16T04:34:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:08755398-e73c-4c46-af1d-b1301f096815", + "resource": { + "resourceType": "Procedure", + "id": "08755398-e73c-4c46-af1d-b1301f096815", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" + }, + "performedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:34:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:fefee1b5-c6c1-4b6c-9a4b-9242043c93d0", + "resource": { + "resourceType": "Claim", + "id": "fefee1b5-c6c1-4b6c-9a4b-9242043c93d0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-16T04:34:51-08:00" + }, + "created": "2016-02-16T04:34:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:08755398-e73c-4c46-af1d-b1301f096815" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:074620ab-31a2-4434-8cca-e14950d0b4d1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "074620ab-31a2-4434-8cca-e14950d0b4d1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fefee1b5-c6c1-4b6c-9a4b-9242043c93d0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-02-16T04:34:51-08:00", + "end": "2017-02-16T04:34:51-08:00" + }, + "created": "2016-02-16T04:34:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:fefee1b5-c6c1-4b6c-9a4b-9242043c93d0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-16T04:34:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-16T04:34:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68", + "resource": { + "resourceType": "Encounter", + "id": "1d625271-3a13-4eab-a560-641b010fab68", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:53:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:53:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:65458bc2-4bb3-49ac-bd7a-756719221cc1", + "resource": { + "resourceType": "Observation", + "id": "65458bc2-4bb3-49ac-bd7a-756719221cc1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "therapy", + "display": "therapy" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "88040-1", + "display": "Response to cancer treatment" + } + ], + "text": "Response to cancer treatment" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" + }, + "effectiveDateTime": "2016-02-15T04:15:51-08:00", + "issued": "2016-02-15T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385633008", + "display": "Improving (qualifier value)" + } + ], + "text": "Improving (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff415025-4fdc-4dad-95c7-c7e03d8ee7da", + "resource": { + "resourceType": "Procedure", + "id": "ff415025-4fdc-4dad-95c7-c7e03d8ee7da", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" + }, + "performedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:38:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:a0e401f7-f75d-4979-8eeb-6f64d42a2989", + "resource": { + "resourceType": "Claim", + "id": "a0e401f7-f75d-4979-8eeb-6f64d42a2989", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:53:51-08:00" + }, + "created": "2016-02-15T04:53:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ff415025-4fdc-4dad-95c7-c7e03d8ee7da" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 10988.12, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b810d0ee-155b-4966-b24f-3e1d78836faf", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b810d0ee-155b-4966-b24f-3e1d78836faf", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a0e401f7-f75d-4979-8eeb-6f64d42a2989" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-02-15T04:53:51-08:00", + "end": "2017-02-15T04:53:51-08:00" + }, + "created": "2016-02-15T04:53:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:a0e401f7-f75d-4979-8eeb-6f64d42a2989" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:53:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2016-02-15T04:15:51-08:00", + "end": "2016-02-15T04:53:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 10988.12, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2197.6240000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 8790.496000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 10988.12, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 10988.12, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 8790.496000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51", + "resource": { + "resourceType": "Encounter", + "id": "25239180-f721-4740-92d9-5431876d9b51", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-06-06T05:15:51-07:00", + "end": "2016-06-06T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-06-06T05:15:51-07:00", + "end": "2016-06-06T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7241d53f-c28d-40f6-87e8-9d13adb9e3c3", + "resource": { + "resourceType": "Claim", + "id": "7241d53f-c28d-40f6-87e8-9d13adb9e3c3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-06-06T05:15:51-07:00", + "end": "2016-06-06T05:30:51-07:00" + }, + "created": "2016-06-06T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a40eaa1d-53b2-4ced-9701-f0f0eebf44c9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a40eaa1d-53b2-4ced-9701-f0f0eebf44c9", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7241d53f-c28d-40f6-87e8-9d13adb9e3c3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-06-06T05:30:51-07:00", + "end": "2017-06-06T05:30:51-07:00" + }, + "created": "2016-06-06T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7241d53f-c28d-40f6-87e8-9d13adb9e3c3" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2016-06-06T05:15:51-07:00", + "end": "2016-06-06T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6", + "resource": { + "resourceType": "Encounter", + "id": "73b11689-39d9-4c07-92a2-fcd6f137d5d6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-08-01T05:15:51-07:00", + "end": "2016-08-01T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-08-01T05:15:51-07:00", + "end": "2016-08-01T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1", + "resource": { + "resourceType": "Condition", + "id": "3af34e51-849b-479a-83c6-ca3b9263a9f1", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" + }, + "onsetDateTime": "2016-08-01T05:15:51-07:00", + "abatementDateTime": "2016-08-08T05:15:51-07:00", + "recordedDate": "2016-08-01T05:15:51-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:bc0d8224-b2fc-403b-8dca-021565a22dfa", + "resource": { + "resourceType": "Claim", + "id": "bc0d8224-b2fc-403b-8dca-021565a22dfa", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-08-01T05:15:51-07:00", + "end": "2016-08-01T05:30:51-07:00" + }, + "created": "2016-08-01T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b8873578-29eb-4ab2-9695-ef618506e0fd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b8873578-29eb-4ab2-9695-ef618506e0fd", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bc0d8224-b2fc-403b-8dca-021565a22dfa" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-08-01T05:30:51-07:00", + "end": "2017-08-01T05:30:51-07:00" + }, + "created": "2016-08-01T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:bc0d8224-b2fc-403b-8dca-021565a22dfa" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2016-08-01T05:15:51-07:00", + "end": "2016-08-01T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "servicedPeriod": { + "start": "2016-08-01T05:15:51-07:00", + "end": "2016-08-01T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60", + "resource": { + "resourceType": "Encounter", + "id": "44a8b8e0-4109-40bb-969a-9a2d23b56c60", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-10-03T05:15:51-07:00", + "end": "2016-10-03T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2016-10-03T05:15:51-07:00", + "end": "2016-10-03T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:60f58125-977a-4d08-9be3-42f6fe8b8c47", + "resource": { + "resourceType": "Claim", + "id": "60f58125-977a-4d08-9be3-42f6fe8b8c47", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-10-03T05:15:51-07:00", + "end": "2016-10-03T05:30:51-07:00" + }, + "created": "2016-10-03T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9a675af5-4f3a-4c26-885a-1b72ccd1bead", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9a675af5-4f3a-4c26-885a-1b72ccd1bead", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "60f58125-977a-4d08-9be3-42f6fe8b8c47" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-10-03T05:30:51-07:00", + "end": "2017-10-03T05:30:51-07:00" + }, + "created": "2016-10-03T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:60f58125-977a-4d08-9be3-42f6fe8b8c47" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2016-10-03T05:15:51-07:00", + "end": "2016-10-03T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6", + "resource": { + "resourceType": "Encounter", + "id": "be05b886-37df-4c36-8bf4-7eab3e9445d6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f12d620d-10c3-4475-b22b-d623a62fc2de", + "resource": { + "resourceType": "Observation", + "id": "f12d620d-10c3-4475-b22b-d623a62fc2de", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f8f7b163-3c42-4424-83f3-d77dd0270661", + "resource": { + "resourceType": "Observation", + "id": "f8f7b163-3c42-4424-83f3-d77dd0270661", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9b1f42ff-1f95-40fb-8c68-2dd5e9e73e1f", + "resource": { + "resourceType": "Observation", + "id": "9b1f42ff-1f95-40fb-8c68-2dd5e9e73e1f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e971fcff-4d62-4fd0-bdb3-9d141040b0fa", + "resource": { + "resourceType": "Observation", + "id": "e971fcff-4d62-4fd0-bdb3-9d141040b0fa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:28fc98fb-2954-4288-b519-1ab918ca827d", + "resource": { + "resourceType": "Observation", + "id": "28fc98fb-2954-4288-b519-1ab918ca827d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 115, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:72542d1b-bdce-4d3a-b964-db3ee94d8e56", + "resource": { + "resourceType": "Observation", + "id": "72542d1b-bdce-4d3a-b964-db3ee94d8e56", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 82, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71f64330-92fa-47c4-b557-e15e2b2f6443", + "resource": { + "resourceType": "Observation", + "id": "71f64330-92fa-47c4-b557-e15e2b2f6443", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:46799724-940f-4725-94a3-46c4d00515d2", + "resource": { + "resourceType": "Observation", + "id": "46799724-940f-4725-94a3-46c4d00515d2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 92.39, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5827a700-4e2a-4107-8661-170a932d388a", + "resource": { + "resourceType": "Observation", + "id": "5827a700-4e2a-4107-8661-170a932d388a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 8.45, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f36fadcb-e8d2-4599-bde6-b08f1296d074", + "resource": { + "resourceType": "Observation", + "id": "f36fadcb-e8d2-4599-bde6-b08f1296d074", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.79, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e8e933d-db08-4965-953a-df40d7bbc79a", + "resource": { + "resourceType": "Observation", + "id": "7e8e933d-db08-4965-953a-df40d7bbc79a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.39, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92b787a5-30dc-4591-b549-89479c5bdce5", + "resource": { + "resourceType": "Observation", + "id": "92b787a5-30dc-4591-b549-89479c5bdce5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 139.34, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f009654-9ec6-4c18-be61-35c587764ab9", + "resource": { + "resourceType": "Observation", + "id": "1f009654-9ec6-4c18-be61-35c587764ab9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.93, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f118ef35-5f46-4ce5-b38b-82064b1cdf79", + "resource": { + "resourceType": "Observation", + "id": "f118ef35-5f46-4ce5-b38b-82064b1cdf79", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 106.58, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c07a523-7bb1-407f-a508-f1b29078f23d", + "resource": { + "resourceType": "Observation", + "id": "3c07a523-7bb1-407f-a508-f1b29078f23d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.72, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:09baf491-bdd4-431a-b345-92cbc6715dac", + "resource": { + "resourceType": "Observation", + "id": "09baf491-bdd4-431a-b345-92cbc6715dac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 197.98, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e0127580-1d18-49d9-af8d-aecec758772b", + "resource": { + "resourceType": "Observation", + "id": "e0127580-1d18-49d9-af8d-aecec758772b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 100.79, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3ac0f342-0954-4f2a-b788-d707473739e5", + "resource": { + "resourceType": "Observation", + "id": "3ac0f342-0954-4f2a-b788-d707473739e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 117.9, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bb68cecc-4923-48f6-abe8-d3021562bab1", + "resource": { + "resourceType": "Observation", + "id": "bb68cecc-4923-48f6-abe8-d3021562bab1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 59.93, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:58016733-a0f3-4e54-9c8e-3d4013b253e7", + "resource": { + "resourceType": "Observation", + "id": "58016733-a0f3-4e54-9c8e-3d4013b253e7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a3376cf5-4e9a-4426-bf87-704d6957cccc", + "resource": { + "resourceType": "Observation", + "id": "a3376cf5-4e9a-4426-bf87-704d6957cccc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.01, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f73f5a63-ab01-4f47-aa18-5a94efa4cdba", + "resource": { + "resourceType": "MedicationRequest", + "id": "f73f5a63-ab01-4f47-aa18-5a94efa4cdba", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "authoredOn": "2016-11-21T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a7ccd4d9-c1ef-4867-80b3-2448efb6801a", + "resource": { + "resourceType": "Claim", + "id": "a7ccd4d9-c1ef-4867-80b3-2448efb6801a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "created": "2016-11-21T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:f73f5a63-ab01-4f47-aa18-5a94efa4cdba" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d7c91657-fdf4-48c5-b0ab-78775a66d86f", + "resource": { + "resourceType": "Immunization", + "id": "d7c91657-fdf4-48c5-b0ab-78775a66d86f", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "occurrenceDateTime": "2016-11-21T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:3886cfff-344f-4a97-9bdc-b8f30c5302bc", + "resource": { + "resourceType": "DiagnosticReport", + "id": "3886cfff-344f-4a97-9bdc-b8f30c5302bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:46799724-940f-4725-94a3-46c4d00515d2", + "display": "Glucose" + }, + { + "reference": "urn:uuid:5827a700-4e2a-4107-8661-170a932d388a", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:f36fadcb-e8d2-4599-bde6-b08f1296d074", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:7e8e933d-db08-4965-953a-df40d7bbc79a", + "display": "Calcium" + }, + { + "reference": "urn:uuid:92b787a5-30dc-4591-b549-89479c5bdce5", + "display": "Sodium" + }, + { + "reference": "urn:uuid:1f009654-9ec6-4c18-be61-35c587764ab9", + "display": "Potassium" + }, + { + "reference": "urn:uuid:f118ef35-5f46-4ce5-b38b-82064b1cdf79", + "display": "Chloride" + }, + { + "reference": "urn:uuid:3c07a523-7bb1-407f-a508-f1b29078f23d", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:aa99e6d6-9cce-4d56-b3b3-4439ed011984", + "resource": { + "resourceType": "DiagnosticReport", + "id": "aa99e6d6-9cce-4d56-b3b3-4439ed011984", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + }, + "effectiveDateTime": "2016-11-21T04:15:51-08:00", + "issued": "2016-11-21T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:09baf491-bdd4-431a-b345-92cbc6715dac", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:e0127580-1d18-49d9-af8d-aecec758772b", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:3ac0f342-0954-4f2a-b788-d707473739e5", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:bb68cecc-4923-48f6-abe8-d3021562bab1", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7d45ec7c-d373-442b-aa09-7dd7fabcaabe", + "resource": { + "resourceType": "Claim", + "id": "7d45ec7c-d373-442b-aa09-7dd7fabcaabe", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "created": "2016-11-21T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d7c91657-fdf4-48c5-b0ab-78775a66d86f" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4e25f12e-53da-4b67-afa1-6220bf889e74", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4e25f12e-53da-4b67-afa1-6220bf889e74", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7d45ec7c-d373-442b-aa09-7dd7fabcaabe" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2016-11-21T04:30:51-08:00", + "end": "2017-11-21T04:30:51-08:00" + }, + "created": "2016-11-21T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7d45ec7c-d373-442b-aa09-7dd7fabcaabe" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2016-11-21T04:15:51-08:00", + "end": "2016-11-21T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a", + "resource": { + "resourceType": "Encounter", + "id": "4d1849a6-ff4c-418b-bd72-b524d972018a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f44d668d-3307-4183-9e84-d4017e7d8c5f", + "resource": { + "resourceType": "Claim", + "id": "f44d668d-3307-4183-9e84-d4017e7d8c5f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:30:51-08:00" + }, + "created": "2017-01-23T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:791b968f-c30e-4d52-bb9a-ae25bd424dc0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "791b968f-c30e-4d52-bb9a-ae25bd424dc0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f44d668d-3307-4183-9e84-d4017e7d8c5f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-01-23T04:30:51-08:00", + "end": "2018-01-23T04:30:51-08:00" + }, + "created": "2017-01-23T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:f44d668d-3307-4183-9e84-d4017e7d8c5f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40", + "resource": { + "resourceType": "Encounter", + "id": "c528e256-7619-4a5f-8384-48db8c7a1e40", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-24T04:38:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-24T04:38:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e0724ed8-53b9-4752-9840-9921e99271b3", + "resource": { + "resourceType": "Procedure", + "id": "e0724ed8-53b9-4752-9840-9921e99271b3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" + }, + "performedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:38:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:cd880169-5053-4258-bc4a-0b4189326536", + "resource": { + "resourceType": "Claim", + "id": "cd880169-5053-4258-bc4a-0b4189326536", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-24T04:38:51-08:00" + }, + "created": "2017-01-24T04:38:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e0724ed8-53b9-4752-9840-9921e99271b3" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:33713f2f-3760-4c70-8191-4efc896f963d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "33713f2f-3760-4c70-8191-4efc896f963d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cd880169-5053-4258-bc4a-0b4189326536" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-01-24T04:38:51-08:00", + "end": "2018-01-24T04:38:51-08:00" + }, + "created": "2017-01-24T04:38:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:cd880169-5053-4258-bc4a-0b4189326536" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-24T04:38:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-24T04:38:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1", + "resource": { + "resourceType": "Encounter", + "id": "71cd8fee-25f0-44c6-8aa6-bb0031af72a1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:41:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:41:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:960d5783-e2e4-4692-a233-1233300e816b", + "resource": { + "resourceType": "Observation", + "id": "960d5783-e2e4-4692-a233-1233300e816b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "therapy", + "display": "therapy" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "88040-1", + "display": "Response to cancer treatment" + } + ], + "text": "Response to cancer treatment" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" + }, + "effectiveDateTime": "2017-01-23T04:15:51-08:00", + "issued": "2017-01-23T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385633008", + "display": "Improving (qualifier value)" + } + ], + "text": "Improving (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a524a6e8-00f8-41f2-b1c4-3238d2e7ed45", + "resource": { + "resourceType": "Procedure", + "id": "a524a6e8-00f8-41f2-b1c4-3238d2e7ed45", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" + }, + "performedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:26:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0cbd621b-20ec-4d1b-b955-4e8e3d7c8375", + "resource": { + "resourceType": "Claim", + "id": "0cbd621b-20ec-4d1b-b955-4e8e3d7c8375", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:41:51-08:00" + }, + "created": "2017-01-23T04:41:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a524a6e8-00f8-41f2-b1c4-3238d2e7ed45" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 10282.89, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:03603520-d43f-479e-86be-5926a6114970", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "03603520-d43f-479e-86be-5926a6114970", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0cbd621b-20ec-4d1b-b955-4e8e3d7c8375" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-01-23T04:41:51-08:00", + "end": "2018-01-23T04:41:51-08:00" + }, + "created": "2017-01-23T04:41:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0cbd621b-20ec-4d1b-b955-4e8e3d7c8375" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:41:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2017-01-23T04:15:51-08:00", + "end": "2017-01-23T04:41:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 10282.89, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2056.578, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 8226.312, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 10282.89, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 10282.89, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 8226.312, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8", + "resource": { + "resourceType": "Encounter", + "id": "921a28d6-913a-472c-88a7-dfb9b45895e8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-05-13T05:15:51-07:00", + "end": "2017-05-13T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-05-13T05:15:51-07:00", + "end": "2017-05-13T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29", + "resource": { + "resourceType": "Condition", + "id": "7aeb9bc7-24f9-4145-b5b3-15be17203a29", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" + }, + "onsetDateTime": "2017-05-13T05:15:51-07:00", + "abatementDateTime": "2017-06-03T05:15:51-07:00", + "recordedDate": "2017-05-13T05:15:51-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b5b01012-a227-41dc-89f5-e2485647ece0", + "resource": { + "resourceType": "Claim", + "id": "b5b01012-a227-41dc-89f5-e2485647ece0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-05-13T05:15:51-07:00", + "end": "2017-05-13T05:30:51-07:00" + }, + "created": "2017-05-13T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fe34960b-1bf7-473e-857a-d644fb405805", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "fe34960b-1bf7-473e-857a-d644fb405805", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b5b01012-a227-41dc-89f5-e2485647ece0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-05-13T05:30:51-07:00", + "end": "2018-05-13T05:30:51-07:00" + }, + "created": "2017-05-13T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b5b01012-a227-41dc-89f5-e2485647ece0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2017-05-13T05:15:51-07:00", + "end": "2017-05-13T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "servicedPeriod": { + "start": "2017-05-13T05:15:51-07:00", + "end": "2017-05-13T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf", + "resource": { + "resourceType": "Encounter", + "id": "9d6e40c7-408e-4375-8830-e0b61faa8baf", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-05-22T05:15:51-07:00", + "end": "2017-05-22T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-05-22T05:15:51-07:00", + "end": "2017-05-22T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c0eb680d-2f42-445d-b3a5-519d5c1fa3cc", + "resource": { + "resourceType": "Claim", + "id": "c0eb680d-2f42-445d-b3a5-519d5c1fa3cc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-05-22T05:15:51-07:00", + "end": "2017-05-22T05:30:51-07:00" + }, + "created": "2017-05-22T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b610b97f-954f-4a50-8036-51c8557ce56b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b610b97f-954f-4a50-8036-51c8557ce56b", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c0eb680d-2f42-445d-b3a5-519d5c1fa3cc" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-05-22T05:30:51-07:00", + "end": "2018-05-22T05:30:51-07:00" + }, + "created": "2017-05-22T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:c0eb680d-2f42-445d-b3a5-519d5c1fa3cc" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-05-22T05:15:51-07:00", + "end": "2017-05-22T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb", + "resource": { + "resourceType": "Encounter", + "id": "a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-08-25T05:15:51-07:00", + "end": "2017-08-25T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-08-25T05:15:51-07:00", + "end": "2017-08-25T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488", + "resource": { + "resourceType": "Condition", + "id": "e086a3ee-b70c-4698-9d6d-0de40fa81488", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" + }, + "onsetDateTime": "2017-08-25T05:15:51-07:00", + "abatementDateTime": "2017-09-06T05:15:51-07:00", + "recordedDate": "2017-08-25T05:15:51-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:8ca17759-9c08-4f83-93a8-5e2346524b2a", + "resource": { + "resourceType": "Observation", + "id": "8ca17759-9c08-4f83-93a8-5e2346524b2a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + }, + { + "system": "http://loinc.org", + "code": "8331-1", + "display": "Oral temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" + }, + "effectiveDateTime": "2017-08-25T05:15:51-07:00", + "issued": "2017-08-25T05:15:51.927-07:00", + "valueQuantity": { + "value": 37.861, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd8448c1-d0ef-482d-a44a-33976a57dbac", + "resource": { + "resourceType": "Claim", + "id": "dd8448c1-d0ef-482d-a44a-33976a57dbac", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-08-25T05:15:51-07:00", + "end": "2017-08-25T05:30:51-07:00" + }, + "created": "2017-08-25T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2d081651-0596-4aa6-b209-8935ae2c7e19", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2d081651-0596-4aa6-b209-8935ae2c7e19", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "dd8448c1-d0ef-482d-a44a-33976a57dbac" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-08-25T05:30:51-07:00", + "end": "2018-08-25T05:30:51-07:00" + }, + "created": "2017-08-25T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:dd8448c1-d0ef-482d-a44a-33976a57dbac" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2017-08-25T05:15:51-07:00", + "end": "2017-08-25T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "servicedPeriod": { + "start": "2017-08-25T05:15:51-07:00", + "end": "2017-08-25T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246", + "resource": { + "resourceType": "Encounter", + "id": "25cffaa1-5504-429d-b23c-396d0c723246", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-09-04T05:15:51-07:00", + "end": "2017-09-04T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-09-04T05:15:51-07:00", + "end": "2017-09-04T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5b562284-6d0c-409f-bfe6-290e5d6eab6a", + "resource": { + "resourceType": "Claim", + "id": "5b562284-6d0c-409f-bfe6-290e5d6eab6a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-09-04T05:15:51-07:00", + "end": "2017-09-04T05:30:51-07:00" + }, + "created": "2017-09-04T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5da29a20-e52a-4e25-90ad-71e3e42209a0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5da29a20-e52a-4e25-90ad-71e3e42209a0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5b562284-6d0c-409f-bfe6-290e5d6eab6a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-09-04T05:30:51-07:00", + "end": "2018-09-04T05:30:51-07:00" + }, + "created": "2017-09-04T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5b562284-6d0c-409f-bfe6-290e5d6eab6a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-09-04T05:15:51-07:00", + "end": "2017-09-04T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652", + "resource": { + "resourceType": "Encounter", + "id": "4fcfb59a-0a02-457e-ac45-d3a1c564f652", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:77e9073d-f4d8-4074-a8d5-bbae0463c764", + "resource": { + "resourceType": "Observation", + "id": "77e9073d-f4d8-4074-a8d5-bbae0463c764", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99a8dd5e-deda-4d0b-bf24-c9a8e106ced7", + "resource": { + "resourceType": "Observation", + "id": "99a8dd5e-deda-4d0b-bf24-c9a8e106ced7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7734cb03-d5fd-496b-a840-c0e8fe85a044", + "resource": { + "resourceType": "Observation", + "id": "7734cb03-d5fd-496b-a840-c0e8fe85a044", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:038ea4d5-4c60-4e83-a41f-0d9405e68f2a", + "resource": { + "resourceType": "Observation", + "id": "038ea4d5-4c60-4e83-a41f-0d9405e68f2a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ec9e025-406d-4d43-a7aa-4f66e79646a0", + "resource": { + "resourceType": "Observation", + "id": "0ec9e025-406d-4d43-a7aa-4f66e79646a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 87, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 112, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7766f97-882d-4d06-aa97-a4e081fe66df", + "resource": { + "resourceType": "Observation", + "id": "a7766f97-882d-4d06-aa97-a4e081fe66df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bba53133-ea29-44fa-bb1c-602ecfc7c47e", + "resource": { + "resourceType": "Observation", + "id": "bba53133-ea29-44fa-bb1c-602ecfc7c47e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11bfc228-f8eb-432a-8e85-5409e878e081", + "resource": { + "resourceType": "Observation", + "id": "11bfc228-f8eb-432a-8e85-5409e878e081", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 67.48, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a13f06da-9099-429a-921b-e068b358985a", + "resource": { + "resourceType": "Observation", + "id": "a13f06da-9099-429a-921b-e068b358985a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 12.39, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d379bd6a-cea1-4df3-ade1-c632ba84c235", + "resource": { + "resourceType": "Observation", + "id": "d379bd6a-cea1-4df3-ade1-c632ba84c235", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.78, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:abfa4065-b023-4821-a2a1-ddb68dcb2e7a", + "resource": { + "resourceType": "Observation", + "id": "abfa4065-b023-4821-a2a1-ddb68dcb2e7a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.47, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b13e310-cbb1-4073-83b7-0b14c5210ef4", + "resource": { + "resourceType": "Observation", + "id": "6b13e310-cbb1-4073-83b7-0b14c5210ef4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 142.01, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba524569-514a-4172-87a4-158328c244e6", + "resource": { + "resourceType": "Observation", + "id": "ba524569-514a-4172-87a4-158328c244e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.23, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25d75864-1daa-4b87-a322-7a888027eba3", + "resource": { + "resourceType": "Observation", + "id": "25d75864-1daa-4b87-a322-7a888027eba3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 108.44, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11719ea0-6348-4c70-9574-61da59491254", + "resource": { + "resourceType": "Observation", + "id": "11719ea0-6348-4c70-9574-61da59491254", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.18, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7f9d0f76-b695-4272-a365-7101faf9fc28", + "resource": { + "resourceType": "Observation", + "id": "7f9d0f76-b695-4272-a365-7101faf9fc28", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82731ca9-5a30-4b7c-b8ea-0214e78ae61a", + "resource": { + "resourceType": "Observation", + "id": "82731ca9-5a30-4b7c-b8ea-0214e78ae61a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.26, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ec7c769d-e8d1-47ad-8919-6c1f568e9499", + "resource": { + "resourceType": "MedicationRequest", + "id": "ec7c769d-e8d1-47ad-8919-6c1f568e9499", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "authoredOn": "2017-11-27T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:15b3b06e-717c-443f-a1af-695d9d17f765", + "resource": { + "resourceType": "Claim", + "id": "15b3b06e-717c-443f-a1af-695d9d17f765", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "created": "2017-11-27T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ec7c769d-e8d1-47ad-8919-6c1f568e9499" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7156756a-5620-4e70-b0e3-3b73fe5fec8f", + "resource": { + "resourceType": "Immunization", + "id": "7156756a-5620-4e70-b0e3-3b73fe5fec8f", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "occurrenceDateTime": "2017-11-27T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:5075e05d-7aff-413c-b816-dfa3ead366f3", + "resource": { + "resourceType": "DiagnosticReport", + "id": "5075e05d-7aff-413c-b816-dfa3ead366f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + }, + "effectiveDateTime": "2017-11-27T04:15:51-08:00", + "issued": "2017-11-27T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:11bfc228-f8eb-432a-8e85-5409e878e081", + "display": "Glucose" + }, + { + "reference": "urn:uuid:a13f06da-9099-429a-921b-e068b358985a", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:d379bd6a-cea1-4df3-ade1-c632ba84c235", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:abfa4065-b023-4821-a2a1-ddb68dcb2e7a", + "display": "Calcium" + }, + { + "reference": "urn:uuid:6b13e310-cbb1-4073-83b7-0b14c5210ef4", + "display": "Sodium" + }, + { + "reference": "urn:uuid:ba524569-514a-4172-87a4-158328c244e6", + "display": "Potassium" + }, + { + "reference": "urn:uuid:25d75864-1daa-4b87-a322-7a888027eba3", + "display": "Chloride" + }, + { + "reference": "urn:uuid:11719ea0-6348-4c70-9574-61da59491254", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e", + "resource": { + "resourceType": "Claim", + "id": "9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "created": "2017-11-27T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7156756a-5620-4e70-b0e3-3b73fe5fec8f" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4dfb468f-701e-499b-b17f-1220670ac9f0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4dfb468f-701e-499b-b17f-1220670ac9f0", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-11-27T04:30:51-08:00", + "end": "2018-11-27T04:30:51-08:00" + }, + "created": "2017-11-27T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2017-11-27T04:15:51-08:00", + "end": "2017-11-27T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee", + "resource": { + "resourceType": "Encounter", + "id": "69459341-de89-4353-8c45-b61275db24ee", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:86d8ad2b-e60b-427c-903e-2804c948e723", + "resource": { + "resourceType": "Claim", + "id": "86d8ad2b-e60b-427c-903e-2804c948e723", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:30:51-08:00" + }, + "created": "2017-12-25T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:52f2d5e5-2e85-4829-a6fa-9e36bc728da3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "52f2d5e5-2e85-4829-a6fa-9e36bc728da3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "86d8ad2b-e60b-427c-903e-2804c948e723" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-12-25T04:30:51-08:00", + "end": "2018-12-25T04:30:51-08:00" + }, + "created": "2017-12-25T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:86d8ad2b-e60b-427c-903e-2804c948e723" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea", + "resource": { + "resourceType": "Encounter", + "id": "defaa09a-4bd0-4e24-a7e8-97d0a0db72ea", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-26T04:38:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-26T04:38:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:af2d564d-b557-418b-bbd2-d510d10d21f1", + "resource": { + "resourceType": "Procedure", + "id": "af2d564d-b557-418b-bbd2-d510d10d21f1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" + }, + "performedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:38:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b232f88e-b5c8-4287-8499-164d5364f144", + "resource": { + "resourceType": "Claim", + "id": "b232f88e-b5c8-4287-8499-164d5364f144", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-26T04:38:51-08:00" + }, + "created": "2017-12-26T04:38:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:af2d564d-b557-418b-bbd2-d510d10d21f1" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4fc74029-6198-448c-b038-055b51276e75", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4fc74029-6198-448c-b038-055b51276e75", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b232f88e-b5c8-4287-8499-164d5364f144" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-12-26T04:38:51-08:00", + "end": "2018-12-26T04:38:51-08:00" + }, + "created": "2017-12-26T04:38:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b232f88e-b5c8-4287-8499-164d5364f144" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-26T04:38:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-26T04:38:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3", + "resource": { + "resourceType": "Encounter", + "id": "ef1e49b1-1818-47f4-8f77-b620a6747bc3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:40:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:40:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c81a4b93-cdad-49cd-a4c2-a5b209da107c", + "resource": { + "resourceType": "Observation", + "id": "c81a4b93-cdad-49cd-a4c2-a5b209da107c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "therapy", + "display": "therapy" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "88040-1", + "display": "Response to cancer treatment" + } + ], + "text": "Response to cancer treatment" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" + }, + "effectiveDateTime": "2017-12-25T04:15:51-08:00", + "issued": "2017-12-25T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385633008", + "display": "Improving (qualifier value)" + } + ], + "text": "Improving (qualifier value)" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba020b52-384f-4119-afa3-bcca171c9333", + "resource": { + "resourceType": "Procedure", + "id": "ba020b52-384f-4119-afa3-bcca171c9333", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" + }, + "performedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:25:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e1963125-2416-46aa-9612-fb0d496d3732", + "resource": { + "resourceType": "Claim", + "id": "e1963125-2416-46aa-9612-fb0d496d3732", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:40:51-08:00" + }, + "created": "2017-12-25T04:40:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ba020b52-384f-4119-afa3-bcca171c9333" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 11061.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:42008aa2-f532-4c51-ab91-99a808ffbc17", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "42008aa2-f532-4c51-ab91-99a808ffbc17", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e1963125-2416-46aa-9612-fb0d496d3732" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2017-12-25T04:40:51-08:00", + "end": "2018-12-25T04:40:51-08:00" + }, + "created": "2017-12-25T04:40:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e1963125-2416-46aa-9612-fb0d496d3732" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:40:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2017-12-25T04:15:51-08:00", + "end": "2017-12-25T04:40:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 11061.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2212.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 8849.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11061.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11061.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 8849.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733", + "resource": { + "resourceType": "Encounter", + "id": "0653ded2-d1f8-49db-a068-f0a35c96c733", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-04-16T05:15:51-07:00", + "end": "2018-04-16T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2018-04-16T05:15:51-07:00", + "end": "2018-04-16T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4243e264-2329-4726-9ac8-369d6bde35bc", + "resource": { + "resourceType": "Claim", + "id": "4243e264-2329-4726-9ac8-369d6bde35bc", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-04-16T05:15:51-07:00", + "end": "2018-04-16T05:30:51-07:00" + }, + "created": "2018-04-16T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a4a52946-7b40-428c-acbd-3a320da54164", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a4a52946-7b40-428c-acbd-3a320da54164", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4243e264-2329-4726-9ac8-369d6bde35bc" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-04-16T05:30:51-07:00", + "end": "2019-04-16T05:30:51-07:00" + }, + "created": "2018-04-16T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:4243e264-2329-4726-9ac8-369d6bde35bc" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2018-04-16T05:15:51-07:00", + "end": "2018-04-16T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a", + "resource": { + "resourceType": "Encounter", + "id": "f8a5cf23-c2b5-40e1-ac08-482b606c7f3a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-08-13T05:15:51-07:00", + "end": "2018-08-13T05:30:51-07:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2018-08-13T05:15:51-07:00", + "end": "2018-08-13T05:30:51-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fbfbb74e-0420-474e-a434-d4db67cc6caf", + "resource": { + "resourceType": "Claim", + "id": "fbfbb74e-0420-474e-a434-d4db67cc6caf", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-08-13T05:15:51-07:00", + "end": "2018-08-13T05:30:51-07:00" + }, + "created": "2018-08-13T05:30:51-07:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:405f7699-9c13-46f7-95bb-380d58969a92", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "405f7699-9c13-46f7-95bb-380d58969a92", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fbfbb74e-0420-474e-a434-d4db67cc6caf" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-08-13T05:30:51-07:00", + "end": "2019-08-13T05:30:51-07:00" + }, + "created": "2018-08-13T05:30:51-07:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:fbfbb74e-0420-474e-a434-d4db67cc6caf" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2018-08-13T05:15:51-07:00", + "end": "2018-08-13T05:30:51-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8", + "resource": { + "resourceType": "Encounter", + "id": "a16a6f17-85b4-45d0-a5a4-d7fb05f508d8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:402eb990-0cef-4cce-9517-9b431a941424", + "resource": { + "resourceType": "Observation", + "id": "402eb990-0cef-4cce-9517-9b431a941424", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0485dc4a-d71b-4085-bc71-cdb343a55f0c", + "resource": { + "resourceType": "Observation", + "id": "0485dc4a-d71b-4085-bc71-cdb343a55f0c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2fa854b4-4bfb-46fe-80e1-9eda5a7cc477", + "resource": { + "resourceType": "Observation", + "id": "2fa854b4-4bfb-46fe-80e1-9eda5a7cc477", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c0d8cdd-edec-40b4-85fc-e396ea3deabb", + "resource": { + "resourceType": "Observation", + "id": "9c0d8cdd-edec-40b4-85fc-e396ea3deabb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:57dda705-a066-4cd2-a415-893aedc2bd41", + "resource": { + "resourceType": "Observation", + "id": "57dda705-a066-4cd2-a415-893aedc2bd41", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 118, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b04bc8fa-ca0d-4eb5-ab62-9c42899e05e3", + "resource": { + "resourceType": "Observation", + "id": "b04bc8fa-ca0d-4eb5-ab62-9c42899e05e3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 86, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:661e4067-649d-4a37-8f02-50d89b47e3f4", + "resource": { + "resourceType": "Observation", + "id": "661e4067-649d-4a37-8f02-50d89b47e3f4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca1df88e-e196-4189-8710-d68b85ade4cd", + "resource": { + "resourceType": "Observation", + "id": "ca1df88e-e196-4189-8710-d68b85ade4cd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 65.59, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9cb10ee-a277-456f-9a67-006a4da59bbf", + "resource": { + "resourceType": "Observation", + "id": "c9cb10ee-a277-456f-9a67-006a4da59bbf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 13.13, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0d36548a-7da4-45df-bf21-fc52e82389bb", + "resource": { + "resourceType": "Observation", + "id": "0d36548a-7da4-45df-bf21-fc52e82389bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.66, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fad38a42-303d-4df9-aaca-d0249af1bc35", + "resource": { + "resourceType": "Observation", + "id": "fad38a42-303d-4df9-aaca-d0249af1bc35", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.04, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dcdfe68f-5eb3-4653-9442-ba70b9a38551", + "resource": { + "resourceType": "Observation", + "id": "dcdfe68f-5eb3-4653-9442-ba70b9a38551", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 139.01, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba5d8607-99df-48dc-a201-a768fa5ee017", + "resource": { + "resourceType": "Observation", + "id": "ba5d8607-99df-48dc-a201-a768fa5ee017", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.07, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4efeccf-3e61-4874-b7ad-8784faa4271f", + "resource": { + "resourceType": "Observation", + "id": "c4efeccf-3e61-4874-b7ad-8784faa4271f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 102.09, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4893f495-9876-40ce-b506-5bbcb46ec280", + "resource": { + "resourceType": "Observation", + "id": "4893f495-9876-40ce-b506-5bbcb46ec280", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 23.05, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69b420d1-9a5a-443b-88bf-0c398faec3a9", + "resource": { + "resourceType": "Observation", + "id": "69b420d1-9a5a-443b-88bf-0c398faec3a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 5.9101, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2be60dc0-d024-4db5-81cb-c582dc15a488", + "resource": { + "resourceType": "Observation", + "id": "2be60dc0-d024-4db5-81cb-c582dc15a488", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 4.6104, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4ae9dd3-d629-4f69-8723-0a51262d5f28", + "resource": { + "resourceType": "Observation", + "id": "c4ae9dd3-d629-4f69-8723-0a51262d5f28", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 16.695, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:244d7604-018b-41d8-a10b-4f200ee438c1", + "resource": { + "resourceType": "Observation", + "id": "244d7604-018b-41d8-a10b-4f200ee438c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 35.366, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:da104bff-2240-49ef-8ed9-eb25010c6f08", + "resource": { + "resourceType": "Observation", + "id": "da104bff-2240-49ef-8ed9-eb25010c6f08", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 90.04, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29fbf375-fe82-466c-acbc-12559dad693c", + "resource": { + "resourceType": "Observation", + "id": "29fbf375-fe82-466c-acbc-12559dad693c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.376, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6a59543c-d6ae-47d8-b519-4abf27ea3295", + "resource": { + "resourceType": "Observation", + "id": "6a59543c-d6ae-47d8-b519-4abf27ea3295", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 35.927, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:09aaec1f-0e02-4c07-a3ac-8b383535b013", + "resource": { + "resourceType": "Observation", + "id": "09aaec1f-0e02-4c07-a3ac-8b383535b013", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 44.427, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e20a3a4-1529-43d4-9196-c6d78cedada6", + "resource": { + "resourceType": "Observation", + "id": "4e20a3a4-1529-43d4-9196-c6d78cedada6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 412.82, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:698b6782-257a-4ed4-807e-d8dc04cc8614", + "resource": { + "resourceType": "Observation", + "id": "698b6782-257a-4ed4-807e-d8dc04cc8614", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 316.13, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d6e6775c-d29f-4152-8d76-27ef115a8248", + "resource": { + "resourceType": "Observation", + "id": "d6e6775c-d29f-4152-8d76-27ef115a8248", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 10.306, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:899c5e79-8be4-4f04-a844-c34f190a81cf", + "resource": { + "resourceType": "Observation", + "id": "899c5e79-8be4-4f04-a844-c34f190a81cf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc9babc9-368e-47e5-9b2d-0f4326b9a209", + "resource": { + "resourceType": "Observation", + "id": "fc9babc9-368e-47e5-9b2d-0f4326b9a209", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.38, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf", + "resource": { + "resourceType": "Procedure", + "id": "d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "performedPeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:30:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:45fce3e2-80c4-4fc3-8719-0fae8df44547", + "resource": { + "resourceType": "MedicationRequest", + "id": "45fce3e2-80c4-4fc3-8719-0fae8df44547", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "authoredOn": "2018-12-03T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ddefe41a-1ca4-4da2-966d-717099da18cd", + "resource": { + "resourceType": "Claim", + "id": "ddefe41a-1ca4-4da2-966d-717099da18cd", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "created": "2018-12-03T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:45fce3e2-80c4-4fc3-8719-0fae8df44547" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d3e0de66-678d-448d-9374-fa13288a2bec", + "resource": { + "resourceType": "Immunization", + "id": "d3e0de66-678d-448d-9374-fa13288a2bec", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "occurrenceDateTime": "2018-12-03T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:268fead9-179f-414f-8452-0a9261f9c790", + "resource": { + "resourceType": "Immunization", + "id": "268fead9-179f-414f-8452-0a9261f9c790", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "occurrenceDateTime": "2018-12-03T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c99c9327-f69f-4057-886e-f75ff2a8dfb8", + "resource": { + "resourceType": "DiagnosticReport", + "id": "c99c9327-f69f-4057-886e-f75ff2a8dfb8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:ca1df88e-e196-4189-8710-d68b85ade4cd", + "display": "Glucose" + }, + { + "reference": "urn:uuid:c9cb10ee-a277-456f-9a67-006a4da59bbf", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:0d36548a-7da4-45df-bf21-fc52e82389bb", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:fad38a42-303d-4df9-aaca-d0249af1bc35", + "display": "Calcium" + }, + { + "reference": "urn:uuid:dcdfe68f-5eb3-4653-9442-ba70b9a38551", + "display": "Sodium" + }, + { + "reference": "urn:uuid:ba5d8607-99df-48dc-a201-a768fa5ee017", + "display": "Potassium" + }, + { + "reference": "urn:uuid:c4efeccf-3e61-4874-b7ad-8784faa4271f", + "display": "Chloride" + }, + { + "reference": "urn:uuid:4893f495-9876-40ce-b506-5bbcb46ec280", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:487a8a4b-8df5-45a1-90e2-3d7844ee4fc5", + "resource": { + "resourceType": "DiagnosticReport", + "id": "487a8a4b-8df5-45a1-90e2-3d7844ee4fc5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + }, + "effectiveDateTime": "2018-12-03T04:15:51-08:00", + "issued": "2018-12-03T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:69b420d1-9a5a-443b-88bf-0c398faec3a9", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:2be60dc0-d024-4db5-81cb-c582dc15a488", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:c4ae9dd3-d629-4f69-8723-0a51262d5f28", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:244d7604-018b-41d8-a10b-4f200ee438c1", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:da104bff-2240-49ef-8ed9-eb25010c6f08", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:29fbf375-fe82-466c-acbc-12559dad693c", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:6a59543c-d6ae-47d8-b519-4abf27ea3295", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:09aaec1f-0e02-4c07-a3ac-8b383535b013", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:4e20a3a4-1529-43d4-9196-c6d78cedada6", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:698b6782-257a-4ed4-807e-d8dc04cc8614", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:d6e6775c-d29f-4152-8d76-27ef115a8248", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:249cbe35-b846-406d-bf40-79838cb34695", + "resource": { + "resourceType": "Claim", + "id": "249cbe35-b846-406d-bf40-79838cb34695", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "created": "2018-12-03T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d3e0de66-678d-448d-9374-fa13288a2bec" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:268fead9-179f-414f-8452-0a9261f9c790" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 690.58, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:edf36a35-d061-4442-b136-cb08d2bdd6ff", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "edf36a35-d061-4442-b136-cb08d2bdd6ff", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "249cbe35-b846-406d-bf40-79838cb34695" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-12-03T04:45:51-08:00", + "end": "2019-12-03T04:45:51-08:00" + }, + "created": "2018-12-03T04:45:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:249cbe35-b846-406d-bf40-79838cb34695" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "servicedPeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-03T04:15:51-08:00", + "end": "2018-12-03T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 690.58, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 138.116, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 552.464, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 690.58, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 690.58, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 777.296, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9", + "resource": { + "resourceType": "Encounter", + "id": "59cfb6a4-1794-48c6-8a71-7c0e405415e9", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cfae25a3-fed2-4f3d-b444-1c9a9562df4c", + "resource": { + "resourceType": "Claim", + "id": "cfae25a3-fed2-4f3d-b444-1c9a9562df4c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:30:51-08:00" + }, + "created": "2018-12-10T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:deb68b0a-0052-4720-950d-ea9a78dbdb99", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "deb68b0a-0052-4720-950d-ea9a78dbdb99", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cfae25a3-fed2-4f3d-b444-1c9a9562df4c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-12-10T04:30:51-08:00", + "end": "2019-12-10T04:30:51-08:00" + }, + "created": "2018-12-10T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:cfae25a3-fed2-4f3d-b444-1c9a9562df4c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d", + "resource": { + "resourceType": "Encounter", + "id": "ac9a985e-dfc3-4315-ad32-4a340be7594d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-11T04:32:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-11T04:32:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:02fb4cc4-ca2a-42d2-9500-ffe141963f19", + "resource": { + "resourceType": "Procedure", + "id": "02fb4cc4-ca2a-42d2-9500-ffe141963f19", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" + }, + "performedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:32:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:0962c5c9-7415-4d7c-859e-240f1608cf24", + "resource": { + "resourceType": "Claim", + "id": "0962c5c9-7415-4d7c-859e-240f1608cf24", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-11T04:32:51-08:00" + }, + "created": "2018-12-11T04:32:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:02fb4cc4-ca2a-42d2-9500-ffe141963f19" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:deb19e37-b06d-4fe4-83f4-338b35237769", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "deb19e37-b06d-4fe4-83f4-338b35237769", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0962c5c9-7415-4d7c-859e-240f1608cf24" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-12-11T04:32:51-08:00", + "end": "2019-12-11T04:32:51-08:00" + }, + "created": "2018-12-11T04:32:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:0962c5c9-7415-4d7c-859e-240f1608cf24" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-11T04:32:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-11T04:32:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10", + "resource": { + "resourceType": "Encounter", + "id": "6b91b276-e281-4052-99e0-ca11a5ad0b10", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:46:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:46:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a", + "resource": { + "resourceType": "Procedure", + "id": "07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" + }, + "performedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:31:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:db5c9b6f-cb0a-465e-a769-7eff0513f754", + "resource": { + "resourceType": "Claim", + "id": "db5c9b6f-cb0a-465e-a769-7eff0513f754", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:46:51-08:00" + }, + "created": "2018-12-10T04:46:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 8977.80, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d3fa5207-39fb-4511-b870-c3a524d0d3d1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d3fa5207-39fb-4511-b870-c3a524d0d3d1", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "db5c9b6f-cb0a-465e-a769-7eff0513f754" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2018-12-10T04:46:51-08:00", + "end": "2019-12-10T04:46:51-08:00" + }, + "created": "2018-12-10T04:46:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:db5c9b6f-cb0a-465e-a769-7eff0513f754" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:46:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2018-12-10T04:15:51-08:00", + "end": "2018-12-10T04:46:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 8977.80, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1795.56, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7182.24, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 8977.80, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 8977.80, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7182.24, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4", + "resource": { + "resourceType": "Encounter", + "id": "02d2ed94-8498-4317-bde6-15685b3922c4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "individual": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + } + } + ], + "period": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:dbe5f86c-7242-46f4-8e9a-a196be32876b", + "resource": { + "resourceType": "Observation", + "id": "dbe5f86c-7242-46f4-8e9a-a196be32876b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 172.4, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ef99c4b-c395-47cd-a493-f3298ca1544b", + "resource": { + "resourceType": "Observation", + "id": "6ef99c4b-c395-47cd-a493-f3298ca1544b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cfad61ac-d7fa-4bf4-8824-0cfc6b995e93", + "resource": { + "resourceType": "Observation", + "id": "cfad61ac-d7fa-4bf4-8824-0cfc6b995e93", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 81.3, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c11cd27-1730-4d64-9139-fcba691154da", + "resource": { + "resourceType": "Observation", + "id": "4c11cd27-1730-4d64-9139-fcba691154da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 27.35, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4ed5825-b78b-4546-b343-ba33bd378d8e", + "resource": { + "resourceType": "Observation", + "id": "c4ed5825-b78b-4546-b343-ba33bd378d8e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 137, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bf2a3404-9340-42ef-9701-4c78f48a538d", + "resource": { + "resourceType": "Observation", + "id": "bf2a3404-9340-42ef-9701-4c78f48a538d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1ade4a8-d866-4aa3-bd64-5a86c3fcc67b", + "resource": { + "resourceType": "Observation", + "id": "f1ade4a8-d866-4aa3-bd64-5a86c3fcc67b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", + "resource": { + "resourceType": "Observation", + "id": "9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2339-0", + "display": "Glucose" + } + ], + "text": "Glucose" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 91.67, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", + "resource": { + "resourceType": "Observation", + "id": "765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6299-2", + "display": "Urea Nitrogen" + } + ], + "text": "Urea Nitrogen" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 10.39, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3f0a762-9343-49c3-8a30-944ba6d2a7eb", + "resource": { + "resourceType": "Observation", + "id": "e3f0a762-9343-49c3-8a30-944ba6d2a7eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "38483-4", + "display": "Creatinine" + } + ], + "text": "Creatinine" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 0.64, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4e78702-bab8-4142-b762-b24634162d40", + "resource": { + "resourceType": "Observation", + "id": "c4e78702-bab8-4142-b762-b24634162d40", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "49765-1", + "display": "Calcium" + } + ], + "text": "Calcium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 9.02, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6abfd13a-c86a-4344-bd60-d8bf64a29c03", + "resource": { + "resourceType": "Observation", + "id": "6abfd13a-c86a-4344-bd60-d8bf64a29c03", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2947-0", + "display": "Sodium" + } + ], + "text": "Sodium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 143.87, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:255ca62e-446c-4fc2-bb77-5d1f25d61f8e", + "resource": { + "resourceType": "Observation", + "id": "255ca62e-446c-4fc2-bb77-5d1f25d61f8e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6298-4", + "display": "Potassium" + } + ], + "text": "Potassium" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 5.09, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:92a1d8c2-3825-4ada-8c64-26d997fed777", + "resource": { + "resourceType": "Observation", + "id": "92a1d8c2-3825-4ada-8c64-26d997fed777", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2069-3", + "display": "Chloride" + } + ], + "text": "Chloride" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 103.9, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6dd8001b-73af-4f87-aab2-6a950cf309a1", + "resource": { + "resourceType": "Observation", + "id": "6dd8001b-73af-4f87-aab2-6a950cf309a1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "20565-8", + "display": "Carbon Dioxide" + } + ], + "text": "Carbon Dioxide" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 28.59, + "unit": "mmol/L", + "system": "http://unitsofmeasure.org", + "code": "mmol/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53abb0b6-550a-427c-9bff-082b1b304257", + "resource": { + "resourceType": "Observation", + "id": "53abb0b6-550a-427c-9bff-082b1b304257", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 176.95, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c74af59-8778-4476-9a7a-c0a9dc570f84", + "resource": { + "resourceType": "Observation", + "id": "1c74af59-8778-4476-9a7a-c0a9dc570f84", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 147.09, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e96b7f96-84cc-4072-aa61-945ec6cfc4b3", + "resource": { + "resourceType": "Observation", + "id": "e96b7f96-84cc-4072-aa61-945ec6cfc4b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 77.51, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3180f15c-d615-487a-88be-94c1091844a6", + "resource": { + "resourceType": "Observation", + "id": "3180f15c-d615-487a-88be-94c1091844a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 70.02, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33ddbc69-bbc9-49f8-a735-ca24076f88a1", + "resource": { + "resourceType": "Observation", + "id": "33ddbc69-bbc9-49f8-a735-ca24076f88a1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:09a92eaa-cd19-4188-880e-627b2c49d282", + "resource": { + "resourceType": "Procedure", + "id": "09a92eaa-cd19-4188-880e-627b2c49d282", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "performedPeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:30:51-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:febfdae2-f92f-4249-8413-80b69b156122", + "resource": { + "resourceType": "MedicationRequest", + "id": "febfdae2-f92f-4249-8413-80b69b156122", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "authoredOn": "2019-12-09T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "display": "Dr. Carlyn477 DuBuque211" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d85d8167-4016-41d1-9c31-756b9929c7a3", + "resource": { + "resourceType": "Claim", + "id": "d85d8167-4016-41d1-9c31-756b9929c7a3", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "created": "2019-12-09T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:febfdae2-f92f-4249-8413-80b69b156122" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1f9834ae-3a62-4f5c-b038-7d6d39e64c9e", + "resource": { + "resourceType": "Immunization", + "id": "1f9834ae-3a62-4f5c-b038-7d6d39e64c9e", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "occurrenceDateTime": "2019-12-09T04:15:51-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4e9ba0d5-b3fd-4054-b06f-52e63ed0a15c", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4e9ba0d5-b3fd-4054-b06f-52e63ed0a15c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "51990-0", + "display": "Basic Metabolic Panel" + } + ], + "text": "Basic Metabolic Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", + "display": "Glucose" + }, + { + "reference": "urn:uuid:765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", + "display": "Urea Nitrogen" + }, + { + "reference": "urn:uuid:e3f0a762-9343-49c3-8a30-944ba6d2a7eb", + "display": "Creatinine" + }, + { + "reference": "urn:uuid:c4e78702-bab8-4142-b762-b24634162d40", + "display": "Calcium" + }, + { + "reference": "urn:uuid:6abfd13a-c86a-4344-bd60-d8bf64a29c03", + "display": "Sodium" + }, + { + "reference": "urn:uuid:255ca62e-446c-4fc2-bb77-5d1f25d61f8e", + "display": "Potassium" + }, + { + "reference": "urn:uuid:92a1d8c2-3825-4ada-8c64-26d997fed777", + "display": "Chloride" + }, + { + "reference": "urn:uuid:6dd8001b-73af-4f87-aab2-6a950cf309a1", + "display": "Carbon Dioxide" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:682ae742-ec78-4b23-a251-bb02dce26c63", + "resource": { + "resourceType": "DiagnosticReport", + "id": "682ae742-ec78-4b23-a251-bb02dce26c63", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "result": [ + { + "reference": "urn:uuid:53abb0b6-550a-427c-9bff-082b1b304257", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:1c74af59-8778-4476-9a7a-c0a9dc570f84", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:e96b7f96-84cc-4072-aa61-945ec6cfc4b3", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:3180f15c-d615-487a-88be-94c1091844a6", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f907c0ed-8084-423e-941f-1983dbc20373", + "resource": { + "resourceType": "Claim", + "id": "f907c0ed-8084-423e-941f-1983dbc20373", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "created": "2019-12-09T04:45:51-08:00", + "provider": { + "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "display": "PCP128586" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1f9834ae-3a62-4f5c-b038-7d6d39e64c9e" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:09a92eaa-cd19-4188-880e-627b2c49d282" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 351.67, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:144a0a57-5508-454e-b497-f1ded7bccd19", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "144a0a57-5508-454e-b497-f1ded7bccd19", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "performer": [ + { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f907c0ed-8084-423e-941f-1983dbc20373" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-09T04:45:51-08:00", + "end": "2020-12-09T04:45:51-08:00" + }, + "created": "2019-12-09T04:45:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:f907c0ed-8084-423e-941f-1983dbc20373" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-09T04:15:51-08:00", + "end": "2019-12-09T04:45:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 351.67, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 70.334, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 281.336, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 351.67, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 351.67, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 393.752, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004", + "resource": { + "resourceType": "Encounter", + "id": "e4c18a30-3495-4590-bc90-c8c213f6f004", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:30:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:30:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:176fbef1-fae8-41dd-9448-77b5c46a154f", + "resource": { + "resourceType": "Claim", + "id": "176fbef1-fae8-41dd-9448-77b5c46a154f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:30:51-08:00" + }, + "created": "2019-12-05T04:30:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c42c40a5-25a0-46dd-a429-51e55bf5c820", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c42c40a5-25a0-46dd-a429-51e55bf5c820", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "176fbef1-fae8-41dd-9448-77b5c46a154f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-05T04:30:51-08:00", + "end": "2020-12-05T04:30:51-08:00" + }, + "created": "2019-12-05T04:30:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:176fbef1-fae8-41dd-9448-77b5c46a154f" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439740005", + "display": "Postoperative follow-up visit (procedure)" + } + ], + "text": "Postoperative follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:30:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2", + "resource": { + "resourceType": "Encounter", + "id": "cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "IMP" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-06T04:39:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-06T04:39:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:be961283-e839-4d04-b38b-deba677062bd", + "resource": { + "resourceType": "Procedure", + "id": "be961283-e839-4d04-b38b-deba677062bd", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" + }, + "performedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:39:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:470b8402-b1df-4cd0-8d7a-03ba12941e61", + "resource": { + "resourceType": "Claim", + "id": "470b8402-b1df-4cd0-8d7a-03ba12941e61", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-06T04:39:51-08:00" + }, + "created": "2019-12-06T04:39:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:be961283-e839-4d04-b38b-deba677062bd" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "encounter": [ + { + "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "net": { + "value": 516.65, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9275a16b-2d24-4c1e-90cc-e7a7085ee9c2", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9275a16b-2d24-4c1e-90cc-e7a7085ee9c2", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "470b8402-b1df-4cd0-8d7a-03ba12941e61" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-06T04:39:51-08:00", + "end": "2020-12-06T04:39:51-08:00" + }, + "created": "2019-12-06T04:39:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:470b8402-b1df-4cd0-8d7a-03ba12941e61" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410410006", + "display": "Screening surveillance (regime/therapy)" + } + ], + "text": "Screening surveillance (regime/therapy)" + }, + "servicedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-06T04:39:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "71651007", + "display": "Mammography (procedure)" + } + ], + "text": "Mammography (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-06T04:39:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 516.65, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 413.32, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608", + "resource": { + "resourceType": "Encounter", + "id": "e7ea7228-a522-422b-92f4-1f1481846608", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Mrs. Robbyn526 DuBuque211" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "individual": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + } + } + ], + "period": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "254837009", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:eb015fef-3373-411e-a4ca-bcf224f1684d", + "resource": { + "resourceType": "Observation", + "id": "eb015fef-3373-411e-a4ca-bcf224f1684d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4548-4", + "display": "Hemoglobin A1c/Hemoglobin.total in Blood" + } + ], + "text": "Hemoglobin A1c/Hemoglobin.total in Blood" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + }, + "effectiveDateTime": "2019-12-09T04:15:51-08:00", + "issued": "2019-12-09T04:15:51.927-08:00", + "valueQuantity": { + "value": 6.34, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff7af662-9815-4168-b804-c2056ba587e2", + "resource": { + "resourceType": "Procedure", + "id": "ff7af662-9815-4168-b804-c2056ba587e2", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + }, + "performedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:36:51-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", + "display": "Malignant neoplasm of breast (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ed2ee0ba-e4eb-4ae0-9962-0933341559e1", + "resource": { + "resourceType": "MedicationRequest", + "id": "ed2ee0ba-e4eb-4ae0-9962-0933341559e1", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "encounter": { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + }, + "authoredOn": "2019-12-09T04:15:51-08:00", + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "display": "Dr. Heath320 Kautzer186" + }, + "reasonReference": [ + { + "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0df140b9-02bf-46a3-8022-1270a62d335f", + "resource": { + "resourceType": "Claim", + "id": "0df140b9-02bf-46a3-8022-1270a62d335f", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "created": "2019-12-05T04:51:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:ed2ee0ba-e4eb-4ae0-9962-0933341559e1" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + } + ] + } + ], + "total": { + "value": 263.49, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5d7891b9-ce88-4018-b029-1734949ed66a", + "resource": { + "resourceType": "Claim", + "id": "5d7891b9-ce88-4018-b029-1734949ed66a", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", + "display": "Robbyn526 DuBuque211" + }, + "billablePeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "created": "2019-12-05T04:51:51-08:00", + "provider": { + "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "display": "CAPE COD HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ff7af662-9815-4168-b804-c2056ba587e2" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "net": { + "value": 13488.18, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3d6162ce-7a2c-4317-b0a4-d568af4b9f90", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3d6162ce-7a2c-4317-b0a4-d568af4b9f90", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "requester": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "performer": [ + { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Medicare" + }, + "beneficiary": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "payor": [ + { + "display": "Medicare" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5d7891b9-ce88-4018-b029-1734949ed66a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" + }, + "billablePeriod": { + "start": "2019-12-05T04:51:51-08:00", + "end": "2020-12-05T04:51:51-08:00" + }, + "created": "2019-12-05T04:51:51-08:00", + "insurer": { + "display": "Medicare" + }, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:5d7891b9-ce88-4018-b029-1734949ed66a" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Medicare" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185389009", + "display": "Follow-up visit (procedure)" + } + ], + "text": "Follow-up visit (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "312681000", + "display": "Bone density scan (procedure)" + } + ], + "text": "Bone density scan (procedure)" + }, + "servicedPeriod": { + "start": "2019-12-05T04:15:51-08:00", + "end": "2019-12-05T04:51:51-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 13488.18, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2697.6360000000004, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 10790.544000000002, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 13488.18, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 13488.18, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 10790.544000000002, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json new file mode 100644 index 000000000000..c4e42e861091 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json @@ -0,0 +1,16018 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "resource": { + "resourceType": "Patient", + "id": "55a5307d-5f23-49c1-9100-7d5c513abca3", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -792204183018886300 Population seed: 1586368870505
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Lan153 Dicki44" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", + "valueAddress": { + "city": "Cologne", + "state": "North Rhine-Westphalia", + "country": "DE" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.1534940636791636 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 51.84650593632084 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "SS", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-65-3064" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99982974" + }, + { + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0203", + "code": "PPN", + "display": "Passport Number" + } + ], + "text": "Passport Number" + }, + "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", + "value": "X56290648X" + } + ], + "name": [ + { + "use": "official", + "family": "Shanahan202", + "given": [ + "Seymour882" + ], + "prefix": [ + "Mr." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-865-2965", + "use": "home" + } + ], + "gender": "male", + "birthDate": "1967-05-14", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.38437875197168 + }, + { + "url": "longitude", + "valueDecimal": -71.06923432925981 + } + ] + } + ], + "line": [ + "713 Hansen Promenade" + ], + "city": "Everett", + "state": "Massachusetts", + "postalCode": "02149", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", + "code": "M", + "display": "M" + } + ], + "text": "M" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "de-DE", + "display": "German (Germany)" + } + ], + "text": "German (Germany)" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "resource": { + "resourceType": "Practitioner", + "id": "eabb178d-5fde-3f2a-b2b0-84b601205578", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999989" + } + ], + "active": true, + "name": [ + { + "family": "Streich926", + "given": [ + "Ja391" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Ja391.Streich926@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede", + "resource": { + "resourceType": "Encounter", + "id": "b85dd21a-4193-44a8-bcc5-f53d16278ede", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-02-28T17:04:38-08:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-02-28T17:04:38-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:21af527f-28ce-41d2-ac17-ba4af06c18a0", + "resource": { + "resourceType": "CareTeam", + "id": "21af527f-28ce-41d2-ac17-ba4af06c18a0", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" + }, + "period": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-04-19T16:32:38-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:267172ad-fda0-4a8f-b5a6-a1725266f55d", + "resource": { + "resourceType": "CarePlan", + "id": "267172ad-fda0-4a8f-b5a6-a1725266f55d", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" + }, + "period": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-04-19T16:32:38-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:21af527f-28ce-41d2-ac17-ba4af06c18a0" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:1f149a81-6393-4202-b62e-6f096f32d053", + "resource": { + "resourceType": "Claim", + "id": "1f149a81-6393-4202-b62e-6f096f32d053", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-02-28T17:04:38-08:00" + }, + "created": "1970-02-28T17:04:38-08:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9fd3fbe8-b7d7-4228-ac55-b4acd544c720", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9fd3fbe8-b7d7-4228-ac55-b4acd544c720", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1f149a81-6393-4202-b62e-6f096f32d053" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "1970-02-28T17:04:38-08:00", + "end": "1971-02-28T17:04:38-08:00" + }, + "created": "1970-02-28T17:04:38-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:1f149a81-6393-4202-b62e-6f096f32d053" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "1970-02-28T16:32:38-08:00", + "end": "1970-02-28T17:04:38-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488", + "resource": { + "resourceType": "Encounter", + "id": "f7e47867-5416-4c93-a31b-ae8f1f0b0488", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-01T16:47:38-08:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-01T16:47:38-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7290ad08-2670-4c83-a355-bdd794e2ba24", + "resource": { + "resourceType": "CareTeam", + "id": "7290ad08-2670-4c83-a355-bdd794e2ba24", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" + }, + "period": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-08T16:32:38-08:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:c78490fd-321a-4d95-a680-6fcaf0eda0c7", + "resource": { + "resourceType": "CarePlan", + "id": "c78490fd-321a-4d95-a680-6fcaf0eda0c7", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" + }, + "period": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-08T16:32:38-08:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:7290ad08-2670-4c83-a355-bdd794e2ba24" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:826ab08e-cf42-4eab-8c79-9bc4991e9549", + "resource": { + "resourceType": "Claim", + "id": "826ab08e-cf42-4eab-8c79-9bc4991e9549", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-01T16:47:38-08:00" + }, + "created": "2002-02-01T16:47:38-08:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:154d5897-8b4c-467d-aaed-767bd032465d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "154d5897-8b4c-467d-aaed-767bd032465d", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "826ab08e-cf42-4eab-8c79-9bc4991e9549" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2002-02-01T16:47:38-08:00", + "end": "2003-02-01T16:47:38-08:00" + }, + "created": "2002-02-01T16:47:38-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:826ab08e-cf42-4eab-8c79-9bc4991e9549" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2002-02-01T16:32:38-08:00", + "end": "2002-02-01T16:47:38-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7", + "resource": { + "resourceType": "Encounter", + "id": "e7a6d528-eb1c-4433-81e0-969a6117f0f7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2002-02-10T16:32:38-08:00", + "end": "2002-02-10T16:47:38-08:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2002-02-10T16:32:38-08:00", + "end": "2002-02-10T16:47:38-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9", + "resource": { + "resourceType": "Condition", + "id": "2fea83b5-42be-43c6-90ff-19b3474404c9", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "active" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" + }, + "onsetDateTime": "2002-02-10T16:32:38-08:00", + "recordedDate": "2002-02-10T16:32:38-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:3ce2c1fb-3037-4243-a36f-615cf940f147", + "resource": { + "resourceType": "Claim", + "id": "3ce2c1fb-3037-4243-a36f-615cf940f147", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2002-02-10T16:32:38-08:00", + "end": "2002-02-10T16:47:38-08:00" + }, + "created": "2002-02-10T16:47:38-08:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7ccdd76f-114e-4f0e-9256-bcf79003d7fd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7ccdd76f-114e-4f0e-9256-bcf79003d7fd", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3ce2c1fb-3037-4243-a36f-615cf940f147" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2002-02-10T16:47:38-08:00", + "end": "2003-02-10T16:47:38-08:00" + }, + "created": "2002-02-10T16:47:38-08:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3ce2c1fb-3037-4243-a36f-615cf940f147" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "2002-02-10T16:32:38-08:00", + "end": "2002-02-10T16:47:38-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162864005", + "display": "Body mass index 30+ - obesity (finding)" + } + ], + "text": "Body mass index 30+ - obesity (finding)" + }, + "servicedPeriod": { + "start": "2002-02-10T16:32:38-08:00", + "end": "2002-02-10T16:47:38-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5", + "resource": { + "resourceType": "Encounter", + "id": "acb018b9-6727-4695-bbd6-cfb1203c8cc5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-04T18:05:38-07:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-04T18:05:38-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2896d320-72d5-41f9-a20b-e0b3167e90fc", + "resource": { + "resourceType": "CareTeam", + "id": "2896d320-72d5-41f9-a20b-e0b3167e90fc", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" + }, + "period": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-11T17:32:38-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:3879053d-fc91-42f2-b08a-dd951e8360d4", + "resource": { + "resourceType": "CarePlan", + "id": "3879053d-fc91-42f2-b08a-dd951e8360d4", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" + }, + "period": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-11T17:32:38-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:2896d320-72d5-41f9-a20b-e0b3167e90fc" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd", + "resource": { + "resourceType": "Claim", + "id": "71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-04T18:05:38-07:00" + }, + "created": "2003-06-04T18:05:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:de0f8980-2c3b-4a18-9c4c-1ad3858eea4f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "de0f8980-2c3b-4a18-9c4c-1ad3858eea4f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2003-06-04T18:05:38-07:00", + "end": "2004-06-04T18:05:38-07:00" + }, + "created": "2003-06-04T18:05:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2003-06-04T17:32:38-07:00", + "end": "2003-06-04T18:05:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551", + "resource": { + "resourceType": "Encounter", + "id": "cc4b50b7-f642-4b8a-b085-dca7290ab551", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2004-11-03T16:52:38-08:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2004-11-03T16:52:38-08:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3c00a98c-9be8-4717-a0b8-ce8c1817a50d", + "resource": { + "resourceType": "CareTeam", + "id": "3c00a98c-9be8-4717-a0b8-ce8c1817a50d", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" + }, + "period": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2007-05-20T17:32:38-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:698109a7-4550-4c9c-ba7a-5d45ca19bb0c", + "resource": { + "resourceType": "CarePlan", + "id": "698109a7-4550-4c9c-ba7a-5d45ca19bb0c", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" + }, + "period": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2007-05-20T17:32:38-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:3c00a98c-9be8-4717-a0b8-ce8c1817a50d" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:43d81a08-a67e-408e-9d9e-d7ff5f3318b0", + "resource": { + "resourceType": "Claim", + "id": "43d81a08-a67e-408e-9d9e-d7ff5f3318b0", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2004-11-03T16:52:38-08:00" + }, + "created": "2004-11-03T16:52:38-08:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" + } + ] + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:99b0cd78-dfd5-4559-a1da-4451f66a01c9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "99b0cd78-dfd5-4559-a1da-4451f66a01c9", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "43d81a08-a67e-408e-9d9e-d7ff5f3318b0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2004-11-03T16:52:38-08:00", + "end": "2005-11-03T16:52:38-08:00" + }, + "created": "2004-11-03T16:52:38-08:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:43d81a08-a67e-408e-9d9e-d7ff5f3318b0" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2004-11-03T16:32:38-08:00", + "end": "2004-11-03T16:52:38-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296", + "resource": { + "resourceType": "Encounter", + "id": "6246c367-6cba-4eab-aae4-3ffe89959296", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", + "resource": { + "resourceType": "Condition", + "id": "dcc3597f-478f-48f3-ac83-cc9767665562", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + }, + "onsetDateTime": "2011-03-31T17:32:38-07:00", + "abatementDateTime": "2011-04-07T17:32:38-07:00", + "recordedDate": "2011-03-31T17:32:38-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:df0dda35-5a8b-452e-8d64-e979f0fae675", + "resource": { + "resourceType": "Procedure", + "id": "df0dda35-5a8b-452e-8d64-e979f0fae675", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + }, + "performedPeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:35:38-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", + "display": "Acute bronchitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7a0bc7a2-a43d-4148-b932-979f71c28b14", + "resource": { + "resourceType": "MedicationRequest", + "id": "7a0bc7a2-a43d-4148-b932-979f71c28b14", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + }, + "authoredOn": "2011-03-31T17:32:38-07:00", + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + }, + "reasonReference": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:cfe5b3d3-3d7c-4e10-a1c8-56375e181d7d", + "resource": { + "resourceType": "Claim", + "id": "cfe5b3d3-3d7c-4e10-a1c8-56375e181d7d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "created": "2011-03-31T17:50:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:7a0bc7a2-a43d-4148-b932-979f71c28b14" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + } + ] + } + ], + "total": { + "value": 6.6, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:83ce868e-2db5-4d10-9f0a-b2f0fd2da042", + "resource": { + "resourceType": "CareTeam", + "id": "83ce868e-2db5-4d10-9f0a-b2f0fd2da042", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + }, + "period": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-05-29T17:32:38-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:e32d95d5-cb65-48a5-9986-cbc861d4c13a", + "resource": { + "resourceType": "CarePlan", + "id": "e32d95d5-cb65-48a5-9986-cbc861d4c13a", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + }, + "period": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-05-29T17:32:38-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:83ce868e-2db5-4d10-9f0a-b2f0fd2da042" + } + ], + "addresses": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:e6c4c2a7-9cb2-45c1-9f16-85a831ff4456", + "resource": { + "resourceType": "Claim", + "id": "e6c4c2a7-9cb2-45c1-9f16-85a831ff4456", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "created": "2011-03-31T17:50:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:df0dda35-5a8b-452e-8d64-e979f0fae675" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "net": { + "value": 9852.22, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3046af09-6075-4fcc-83dd-a4719ba5aa02", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3046af09-6075-4fcc-83dd-a4719ba5aa02", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e6c4c2a7-9cb2-45c1-9f16-85a831ff4456" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-03-31T17:50:38-07:00", + "end": "2012-03-31T17:50:38-07:00" + }, + "created": "2011-03-31T17:50:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:e6c4c2a7-9cb2-45c1-9f16-85a831ff4456" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "servicedPeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "servicedPeriod": { + "start": "2011-03-31T17:32:38-07:00", + "end": "2011-03-31T17:50:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 9852.22, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1970.444, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 7881.776, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 9852.22, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 9852.22, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 7881.776, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "resource": { + "resourceType": "Organization", + "id": "3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "3265f387-6c51-32ee-8f6d-b2a89caa34d5" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN", + "telecom": [ + { + "system": "phone", + "value": "617-884-8300" + } + ], + "address": [ + { + "line": [ + "151 EVERETT AVENUE" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "2150", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", + "resource": { + "resourceType": "Practitioner", + "id": "fe82a096-86b9-344b-b405-bd24b6ad4715", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906709" + } + ], + "active": true, + "name": [ + { + "family": "Hoppe518", + "given": [ + "Devin82" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Devin82.Hoppe518@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "151 EVERETT AVENUE" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "2150", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb", + "resource": { + "resourceType": "Encounter", + "id": "ce658d96-f9dd-420c-86b7-822b01e259cb", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-04-10T17:32:38-07:00", + "end": "2011-04-10T17:47:38-07:00" + }, + "individual": { + "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", + "display": "Dr. Devin82 Hoppe518" + } + } + ], + "period": { + "start": "2011-04-10T17:32:38-07:00", + "end": "2011-04-10T17:47:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "display": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e8f655cf-40f6-4d92-8370-03bb6961b99c", + "resource": { + "resourceType": "Immunization", + "id": "e8f655cf-40f6-4d92-8370-03bb6961b99c", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" + }, + "occurrenceDateTime": "2011-04-10T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:197f75bd-6b12-4317-8c68-f05c9deced7b", + "resource": { + "resourceType": "Claim", + "id": "197f75bd-6b12-4317-8c68-f05c9deced7b", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2011-04-10T17:32:38-07:00", + "end": "2011-04-10T17:47:38-07:00" + }, + "created": "2011-04-10T17:47:38-07:00", + "provider": { + "reference": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "display": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e8f655cf-40f6-4d92-8370-03bb6961b99c" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a71232d3-d98a-4fba-88ef-ad4be18d4142", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a71232d3-d98a-4fba-88ef-ad4be18d4142", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" + }, + "performer": [ + { + "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "197f75bd-6b12-4317-8c68-f05c9deced7b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-04-10T17:47:38-07:00", + "end": "2012-04-10T17:47:38-07:00" + }, + "created": "2011-04-10T17:47:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:197f75bd-6b12-4317-8c68-f05c9deced7b" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "702927004", + "display": "Urgent care clinic (procedure)" + } + ], + "text": "Urgent care clinic (procedure)" + }, + "servicedPeriod": { + "start": "2011-04-10T17:32:38-07:00", + "end": "2011-04-10T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2011-04-10T17:32:38-07:00", + "end": "2011-04-10T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "resource": { + "resourceType": "Organization", + "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAMBRIDGE HEALTH ALLIANCE", + "telecom": [ + { + "system": "phone", + "value": "6176652300" + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "resource": { + "resourceType": "Practitioner", + "id": "e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999959" + } + ], + "active": true, + "name": [ + { + "family": "Harvey63", + "given": [ + "Alla648" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Alla648.Harvey63@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca", + "resource": { + "resourceType": "Encounter", + "id": "85983bde-e67b-407e-97e9-a635fd9accca", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "individual": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "display": "Dr. Alla648 Harvey63" + } + } + ], + "period": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "display": "CAMBRIDGE HEALTH ALLIANCE" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e", + "resource": { + "resourceType": "Condition", + "id": "1e0f6578-019f-4592-a666-a2438a39fd0e", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "39848009", + "display": "Whiplash injury to neck" + } + ], + "text": "Whiplash injury to neck" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + }, + "onsetDateTime": "2011-05-12T17:32:38-07:00", + "abatementDateTime": "2011-06-02T17:32:38-07:00", + "recordedDate": "2011-05-12T17:32:38-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:e570df1d-5083-4b02-b418-b871f10956a5", + "resource": { + "resourceType": "MedicationRequest", + "id": "e570df1d-5083-4b02-b418-b871f10956a5", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + }, + "authoredOn": "2011-05-12T17:32:38-07:00", + "requester": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "display": "Dr. Alla648 Harvey63" + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:28e0d2b8-4c92-482d-862c-fce1dc363f92", + "resource": { + "resourceType": "Claim", + "id": "28e0d2b8-4c92-482d-862c-fce1dc363f92", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "created": "2011-05-12T18:32:38-07:00", + "provider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "display": "CAMBRIDGE HEALTH ALLIANCE" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:e570df1d-5083-4b02-b418-b871f10956a5" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + } + ] + } + ], + "total": { + "value": 7.92, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b", + "resource": { + "resourceType": "CareTeam", + "id": "7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b", + "status": "inactive", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + }, + "period": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-06-02T17:32:38-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "display": "Dr. Alla648 Harvey63" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "display": "CAMBRIDGE HEALTH ALLIANCE" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "39848009", + "display": "Whiplash injury to neck" + } + ], + "text": "Whiplash injury to neck" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "display": "CAMBRIDGE HEALTH ALLIANCE" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:629069fe-9cfa-42d4-8f96-b6e1b5ef8118", + "resource": { + "resourceType": "CarePlan", + "id": "629069fe-9cfa-42d4-8f96-b6e1b5ef8118", + "text": { + "status": "generated", + "div": "
    Care Plan for Musculoskeletal care.
    Activities:
    • Musculoskeletal care
    • Musculoskeletal care

    Care plan is meant to treat Whiplash injury to neck.
    " + }, + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "408869004", + "display": "Musculoskeletal care" + } + ], + "text": "Musculoskeletal care" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + }, + "period": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-06-02T17:32:38-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b" + } + ], + "addresses": [ + { + "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266694003", + "display": "Heat therapy" + } + ], + "text": "Heat therapy" + }, + "status": "completed", + "location": { + "display": "CAMBRIDGE HEALTH ALLIANCE" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "183051005", + "display": "Recommendation to rest" + } + ], + "text": "Recommendation to rest" + }, + "status": "completed", + "location": { + "display": "CAMBRIDGE HEALTH ALLIANCE" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:9ed2138b-f8e2-4c9c-ad68-18f487d5d20d", + "resource": { + "resourceType": "Claim", + "id": "9ed2138b-f8e2-4c9c-ad68-18f487d5d20d", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "created": "2011-05-12T18:32:38-07:00", + "provider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "display": "CAMBRIDGE HEALTH ALLIANCE" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "39848009", + "display": "Whiplash injury to neck" + } + ], + "text": "Whiplash injury to neck" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3a2d8bdb-4406-4172-b753-2d419d6f2bb4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3a2d8bdb-4406-4172-b753-2d419d6f2bb4", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" + }, + "performer": [ + { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9ed2138b-f8e2-4c9c-ad68-18f487d5d20d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-05-12T18:32:38-07:00", + "end": "2012-05-12T18:32:38-07:00" + }, + "created": "2011-05-12T18:32:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:9ed2138b-f8e2-4c9c-ad68-18f487d5d20d" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + }, + "servicedPeriod": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "39848009", + "display": "Whiplash injury to neck" + } + ], + "text": "Whiplash injury to neck" + }, + "servicedPeriod": { + "start": "2011-05-12T17:32:38-07:00", + "end": "2011-05-12T18:32:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "20", + "display": "Urgent Care Facility" + } + ] + } + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 0.0, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "resource": { + "resourceType": "Organization", + "id": "7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "7f7c08c8-1c39-3b26-bd34-5122a003dd91" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP83180", + "telecom": [ + { + "system": "phone", + "value": "617-389-6951" + } + ], + "address": [ + { + "line": [ + "107 FERRY ST" + ], + "city": "EVERETT", + "state": "MA", + "postalCode": "02149-4940", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "resource": { + "resourceType": "Practitioner", + "id": "12810b32-78af-32f0-b85c-8177e3f2e2e2", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999959759" + } + ], + "active": true, + "name": [ + { + "family": "Goldner995", + "given": [ + "Doretta917" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Doretta917.Goldner995@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "107 FERRY ST" + ], + "city": "EVERETT", + "state": "MA", + "postalCode": "02149-4940", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839", + "resource": { + "resourceType": "Encounter", + "id": "168913b6-d1f2-4f0b-a74f-7ac898c14839", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2011-05-29T17:32:38-07:00", + "end": "2011-05-29T17:47:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2011-05-29T17:32:38-07:00", + "end": "2011-05-29T17:47:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6565831f-d203-479b-9376-d77b0ee4234a", + "resource": { + "resourceType": "Observation", + "id": "6565831f-d203-479b-9376-d77b0ee4234a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2af09e1b-6451-43a5-9edc-701e1184fd8d", + "resource": { + "resourceType": "Observation", + "id": "2af09e1b-6451-43a5-9edc-701e1184fd8d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34171b9d-e2fa-4caa-8610-c570df877412", + "resource": { + "resourceType": "Observation", + "id": "34171b9d-e2fa-4caa-8610-c570df877412", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 88.1, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e5215a80-98f0-4644-bb1f-a3f9698e7edb", + "resource": { + "resourceType": "Observation", + "id": "e5215a80-98f0-4644-bb1f-a3f9698e7edb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 30.38, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d09b9013-96f9-4d2b-af3b-00e058f11f92", + "resource": { + "resourceType": "Observation", + "id": "d09b9013-96f9-4d2b-af3b-00e058f11f92", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 107, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7dddbe4-8f34-4938-808a-571ca75b43c6", + "resource": { + "resourceType": "Observation", + "id": "b7dddbe4-8f34-4938-808a-571ca75b43c6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 72, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d3ea5a9d-6a55-4b5c-8d47-0fc57ae0e09a", + "resource": { + "resourceType": "Observation", + "id": "d3ea5a9d-6a55-4b5c-8d47-0fc57ae0e09a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3e50573-0d80-4b82-8068-bccc745935bf", + "resource": { + "resourceType": "Observation", + "id": "e3e50573-0d80-4b82-8068-bccc745935bf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "effectiveDateTime": "2011-05-29T17:32:38-07:00", + "issued": "2011-05-29T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:edb585d8-20f2-441f-a9a7-655ca5cb7b9a", + "resource": { + "resourceType": "Immunization", + "id": "edb585d8-20f2-441f-a9a7-655ca5cb7b9a", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + }, + "occurrenceDateTime": "2011-05-29T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:3f213d14-94fa-41b7-9705-9b02524cc235", + "resource": { + "resourceType": "Claim", + "id": "3f213d14-94fa-41b7-9705-9b02524cc235", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2011-05-29T17:32:38-07:00", + "end": "2011-05-29T17:47:38-07:00" + }, + "created": "2011-05-29T17:47:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:edb585d8-20f2-441f-a9a7-655ca5cb7b9a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ca7edb6e-e0ce-4ff0-8702-f38e789fb7c9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ca7edb6e-e0ce-4ff0-8702-f38e789fb7c9", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3f213d14-94fa-41b7-9705-9b02524cc235" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2011-05-29T17:47:38-07:00", + "end": "2012-05-29T17:47:38-07:00" + }, + "created": "2011-05-29T17:47:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:3f213d14-94fa-41b7-9705-9b02524cc235" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2011-05-29T17:32:38-07:00", + "end": "2011-05-29T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2011-05-29T17:32:38-07:00", + "end": "2011-05-29T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b", + "resource": { + "resourceType": "Encounter", + "id": "0ed37b12-189d-4f77-a476-322d8b02b50b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8fb30fb0-d2fe-463d-a117-d6d9722471a3", + "resource": { + "resourceType": "Observation", + "id": "8fb30fb0-d2fe-463d-a117-d6d9722471a3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d58ef249-de16-4e21-a868-4cee6108f037", + "resource": { + "resourceType": "Observation", + "id": "d58ef249-de16-4e21-a868-4cee6108f037", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed82af46-5e99-49c2-8d97-18126376d387", + "resource": { + "resourceType": "Observation", + "id": "ed82af46-5e99-49c2-8d97-18126376d387", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 82.8, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc96cec5-a405-43ba-90ef-456c4f8dfa7a", + "resource": { + "resourceType": "Observation", + "id": "fc96cec5-a405-43ba-90ef-456c4f8dfa7a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 28.56, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71eb7072-6c18-4e3c-a350-6fe28fc2eb0f", + "resource": { + "resourceType": "Observation", + "id": "71eb7072-6c18-4e3c-a350-6fe28fc2eb0f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba73a834-1e2a-4c2d-bc61-86dceb0144d3", + "resource": { + "resourceType": "Observation", + "id": "ba73a834-1e2a-4c2d-bc61-86dceb0144d3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8093bc65-4c7a-4d08-bc8c-678045750832", + "resource": { + "resourceType": "Observation", + "id": "8093bc65-4c7a-4d08-bc8c-678045750832", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5a807eb-18d7-4732-bc99-603aa0bbb318", + "resource": { + "resourceType": "Observation", + "id": "a5a807eb-18d7-4732-bc99-603aa0bbb318", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 187.47, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d58bc55-be80-4a32-af90-d989672090ec", + "resource": { + "resourceType": "Observation", + "id": "5d58bc55-be80-4a32-af90-d989672090ec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 118.26, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c1516684-18fb-46e5-b22c-8cb5d8abcb3d", + "resource": { + "resourceType": "Observation", + "id": "c1516684-18fb-46e5-b22c-8cb5d8abcb3d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 102.89, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", + "resource": { + "resourceType": "Observation", + "id": "d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 60.93, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ae060af-1729-4b94-983b-8c4b8a9ec7dc", + "resource": { + "resourceType": "Observation", + "id": "2ae060af-1729-4b94-983b-8c4b8a9ec7dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 7.5831, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c43ca49-92da-48ea-9cf5-9ed8fecfc560", + "resource": { + "resourceType": "Observation", + "id": "5c43ca49-92da-48ea-9cf5-9ed8fecfc560", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 4.9759, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:00676671-b269-48dd-9bc8-c62061d3f270", + "resource": { + "resourceType": "Observation", + "id": "00676671-b269-48dd-9bc8-c62061d3f270", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 13.888, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6511d2e-eaba-4215-af31-3ac7fe071181", + "resource": { + "resourceType": "Observation", + "id": "e6511d2e-eaba-4215-af31-3ac7fe071181", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 45.689, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9359b02-c057-4b87-87a5-511feac9b3f5", + "resource": { + "resourceType": "Observation", + "id": "c9359b02-c057-4b87-87a5-511feac9b3f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 90.832, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9039dc90-96d1-4110-a207-163897880da6", + "resource": { + "resourceType": "Observation", + "id": "9039dc90-96d1-4110-a207-163897880da6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 28.772, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7344abd2-7d08-4043-801d-766dbfa2f32d", + "resource": { + "resourceType": "Observation", + "id": "7344abd2-7d08-4043-801d-766dbfa2f32d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 33.495, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eda3d747-7b0b-4c44-b10d-32929eb24e20", + "resource": { + "resourceType": "Observation", + "id": "eda3d747-7b0b-4c44-b10d-32929eb24e20", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 39.944, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3523c655-0212-44a7-9517-0d2a318a5d78", + "resource": { + "resourceType": "Observation", + "id": "3523c655-0212-44a7-9517-0d2a318a5d78", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 345.47, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:941bd060-2882-4cc9-9e13-d29b9822fe4a", + "resource": { + "resourceType": "Observation", + "id": "941bd060-2882-4cc9-9e13-d29b9822fe4a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 317.47, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e7f57e9-d3a4-4604-8c13-f414e224ccdb", + "resource": { + "resourceType": "Observation", + "id": "5e7f57e9-d3a4-4604-8c13-f414e224ccdb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueQuantity": { + "value": 9.6379, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69f13519-8224-4db2-91e4-a625363eac58", + "resource": { + "resourceType": "Observation", + "id": "69f13519-8224-4db2-91e4-a625363eac58", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36b6a81b-eaea-4d57-9f0b-29f96a3e9861", + "resource": { + "resourceType": "Procedure", + "id": "36b6a81b-eaea-4d57-9f0b-29f96a3e9861", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "performedPeriod": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T17:47:38-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5f878a58-a480-4834-9dec-34cb4ede34e6", + "resource": { + "resourceType": "Immunization", + "id": "5f878a58-a480-4834-9dec-34cb4ede34e6", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "occurrenceDateTime": "2013-06-02T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:54d59b25-230c-4666-b1c9-7ffc76c5eaac", + "resource": { + "resourceType": "DiagnosticReport", + "id": "54d59b25-230c-4666-b1c9-7ffc76c5eaac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "result": [ + { + "reference": "urn:uuid:a5a807eb-18d7-4732-bc99-603aa0bbb318", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:5d58bc55-be80-4a32-af90-d989672090ec", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:c1516684-18fb-46e5-b22c-8cb5d8abcb3d", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:1a8aa626-bf14-4a52-aae3-a1d4a3338976", + "resource": { + "resourceType": "DiagnosticReport", + "id": "1a8aa626-bf14-4a52-aae3-a1d4a3338976", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + }, + "effectiveDateTime": "2013-06-02T17:32:38-07:00", + "issued": "2013-06-02T17:32:38.789-07:00", + "result": [ + { + "reference": "urn:uuid:2ae060af-1729-4b94-983b-8c4b8a9ec7dc", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:5c43ca49-92da-48ea-9cf5-9ed8fecfc560", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:00676671-b269-48dd-9bc8-c62061d3f270", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:e6511d2e-eaba-4215-af31-3ac7fe071181", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:c9359b02-c057-4b87-87a5-511feac9b3f5", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:9039dc90-96d1-4110-a207-163897880da6", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:7344abd2-7d08-4043-801d-766dbfa2f32d", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:eda3d747-7b0b-4c44-b10d-32929eb24e20", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:3523c655-0212-44a7-9517-0d2a318a5d78", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:941bd060-2882-4cc9-9e13-d29b9822fe4a", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:5e7f57e9-d3a4-4604-8c13-f414e224ccdb", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:dc89f63d-273b-4fab-bcab-28d24d55e146", + "resource": { + "resourceType": "Claim", + "id": "dc89f63d-273b-4fab-bcab-28d24d55e146", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "created": "2013-06-02T18:02:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:5f878a58-a480-4834-9dec-34cb4ede34e6" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:36b6a81b-eaea-4d57-9f0b-29f96a3e9861" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 423.86, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5f261924-bfa9-45a7-a125-4dadd8ed66ab", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5f261924-bfa9-45a7-a125-4dadd8ed66ab", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "dc89f63d-273b-4fab-bcab-28d24d55e146" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2013-06-02T18:02:38-07:00", + "end": "2014-06-02T18:02:38-07:00" + }, + "created": "2013-06-02T18:02:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:dc89f63d-273b-4fab-bcab-28d24d55e146" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2013-06-02T17:32:38-07:00", + "end": "2013-06-02T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 423.86, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 84.772, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 339.088, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 423.86, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 423.86, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 451.504, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0", + "resource": { + "resourceType": "Encounter", + "id": "7ffed3ba-6e1a-4502-8b6b-163941059ba0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:54c48d97-ee77-4b4f-b7b0-d48e9947ca68", + "resource": { + "resourceType": "Observation", + "id": "54c48d97-ee77-4b4f-b7b0-d48e9947ca68", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4a36425d-9896-4218-90df-269ae990c1ee", + "resource": { + "resourceType": "Observation", + "id": "4a36425d-9896-4218-90df-269ae990c1ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b7fb02b-d701-41e2-90fe-460b353773d8", + "resource": { + "resourceType": "Observation", + "id": "3b7fb02b-d701-41e2-90fe-460b353773d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 84.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e39ac64-944f-4ebf-8a8b-6ceb9811052d", + "resource": { + "resourceType": "Observation", + "id": "0e39ac64-944f-4ebf-8a8b-6ceb9811052d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 29.15, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cc1bec31-bfb0-4acc-9601-9a41000e07fd", + "resource": { + "resourceType": "Observation", + "id": "cc1bec31-bfb0-4acc-9601-9a41000e07fd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b5e8389a-f729-47a7-99ae-a1b2f91154e3", + "resource": { + "resourceType": "Observation", + "id": "b5e8389a-f729-47a7-99ae-a1b2f91154e3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0afd041a-ca38-44ca-83f9-3a5f4f0c9a12", + "resource": { + "resourceType": "Observation", + "id": "0afd041a-ca38-44ca-83f9-3a5f4f0c9a12", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:beb3b42d-35a0-4c8e-ac9c-8e4a92950aa1", + "resource": { + "resourceType": "Observation", + "id": "beb3b42d-35a0-4c8e-ac9c-8e4a92950aa1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "effectiveDateTime": "2015-06-07T17:32:38-07:00", + "issued": "2015-06-07T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e560761e-0142-4e6a-bd11-99cb7313570f", + "resource": { + "resourceType": "Procedure", + "id": "e560761e-0142-4e6a-bd11-99cb7313570f", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "performedPeriod": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T17:47:38-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:196ac3fc-72d4-43cf-9e23-7bdd6a09bdff", + "resource": { + "resourceType": "Immunization", + "id": "196ac3fc-72d4-43cf-9e23-7bdd6a09bdff", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + }, + "occurrenceDateTime": "2015-06-07T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8b2e8d05-124d-43f0-a35e-0a2a086c50ce", + "resource": { + "resourceType": "Claim", + "id": "8b2e8d05-124d-43f0-a35e-0a2a086c50ce", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "created": "2015-06-07T18:02:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:196ac3fc-72d4-43cf-9e23-7bdd6a09bdff" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e560761e-0142-4e6a-bd11-99cb7313570f" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 451.20, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1952734b-fe29-4594-9861-2fc402104c64", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1952734b-fe29-4594-9861-2fc402104c64", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8b2e8d05-124d-43f0-a35e-0a2a086c50ce" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2015-06-07T18:02:38-07:00", + "end": "2016-06-07T18:02:38-07:00" + }, + "created": "2015-06-07T18:02:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:8b2e8d05-124d-43f0-a35e-0a2a086c50ce" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2015-06-07T17:32:38-07:00", + "end": "2015-06-07T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 451.20, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 90.24000000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 360.96000000000004, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 451.20, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 451.20, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 473.37600000000003, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25", + "resource": { + "resourceType": "Encounter", + "id": "f776bcfc-3dbf-498e-abce-2ae011870b25", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:feadcde7-7aec-43fa-b41e-791efb5cbb71", + "resource": { + "resourceType": "Observation", + "id": "feadcde7-7aec-43fa-b41e-791efb5cbb71", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a6763509-d36a-4856-8d13-f92485daa2b4", + "resource": { + "resourceType": "Observation", + "id": "a6763509-d36a-4856-8d13-f92485daa2b4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bfeb41b7-cc6e-41b6-81f5-02ddb0445f09", + "resource": { + "resourceType": "Observation", + "id": "bfeb41b7-cc6e-41b6-81f5-02ddb0445f09", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 87.7, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:483bc0eb-821d-444c-be38-90f064ad3cd2", + "resource": { + "resourceType": "Observation", + "id": "483bc0eb-821d-444c-be38-90f064ad3cd2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 30.24, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b74cd60d-20aa-44b5-9e11-bc581a7c6ed8", + "resource": { + "resourceType": "Observation", + "id": "b74cd60d-20aa-44b5-9e11-bc581a7c6ed8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 132, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84fc8272-3077-4a79-aaae-726283414f89", + "resource": { + "resourceType": "Observation", + "id": "84fc8272-3077-4a79-aaae-726283414f89", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 90, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a2fc9e1-5bec-44e5-a4cf-b9c057b4ad7c", + "resource": { + "resourceType": "Observation", + "id": "3a2fc9e1-5bec-44e5-a4cf-b9c057b4ad7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a9f8b8cb-9156-4017-a41e-4da710ffc426", + "resource": { + "resourceType": "Observation", + "id": "a9f8b8cb-9156-4017-a41e-4da710ffc426", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2093-3", + "display": "Total Cholesterol" + } + ], + "text": "Total Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 160.74, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b9229d8e-f291-4116-8b78-8c53005413f3", + "resource": { + "resourceType": "Observation", + "id": "b9229d8e-f291-4116-8b78-8c53005413f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2571-8", + "display": "Triglycerides" + } + ], + "text": "Triglycerides" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 108.59, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f4e16d54-96c2-4629-ac36-31ace4f6f3a9", + "resource": { + "resourceType": "Observation", + "id": "f4e16d54-96c2-4629-ac36-31ace4f6f3a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "18262-6", + "display": "Low Density Lipoprotein Cholesterol" + } + ], + "text": "Low Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 64.9, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d6b5d8d8-e511-430f-93e7-2d48550b69d8", + "resource": { + "resourceType": "Observation", + "id": "d6b5d8d8-e511-430f-93e7-2d48550b69d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "2085-9", + "display": "High Density Lipoprotein Cholesterol" + } + ], + "text": "High Density Lipoprotein Cholesterol" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueQuantity": { + "value": 74.12, + "unit": "mg/dL", + "system": "http://unitsofmeasure.org", + "code": "mg/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c2698ad-ac28-486b-a4d3-3a1b358735fb", + "resource": { + "resourceType": "Observation", + "id": "7c2698ad-ac28-486b-a4d3-3a1b358735fb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb6a390b-1c12-4d61-bba2-50dea47dfd0f", + "resource": { + "resourceType": "Immunization", + "id": "cb6a390b-1c12-4d61-bba2-50dea47dfd0f", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "occurrenceDateTime": "2017-05-14T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c8b9efc2-90bb-43ad-9836-04a5c6a10735", + "resource": { + "resourceType": "Immunization", + "id": "c8b9efc2-90bb-43ad-9836-04a5c6a10735", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "occurrenceDateTime": "2017-05-14T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1c25e149-5d29-4c64-9d88-f67a78d6a0fb", + "resource": { + "resourceType": "DiagnosticReport", + "id": "1c25e149-5d29-4c64-9d88-f67a78d6a0fb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "57698-3", + "display": "Lipid Panel" + } + ], + "text": "Lipid Panel" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + }, + "effectiveDateTime": "2017-05-14T17:32:38-07:00", + "issued": "2017-05-14T17:32:38.789-07:00", + "result": [ + { + "reference": "urn:uuid:a9f8b8cb-9156-4017-a41e-4da710ffc426", + "display": "Total Cholesterol" + }, + { + "reference": "urn:uuid:b9229d8e-f291-4116-8b78-8c53005413f3", + "display": "Triglycerides" + }, + { + "reference": "urn:uuid:f4e16d54-96c2-4629-ac36-31ace4f6f3a9", + "display": "Low Density Lipoprotein Cholesterol" + }, + { + "reference": "urn:uuid:d6b5d8d8-e511-430f-93e7-2d48550b69d8", + "display": "High Density Lipoprotein Cholesterol" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:076e6cbb-8e89-40b0-87a1-95f307b3bd7e", + "resource": { + "resourceType": "Claim", + "id": "076e6cbb-8e89-40b0-87a1-95f307b3bd7e", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "created": "2017-05-14T17:47:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:cb6a390b-1c12-4d61-bba2-50dea47dfd0f" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c8b9efc2-90bb-43ad-9836-04a5c6a10735" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cc4a346a-b5e5-49cd-a056-f600d538a65f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cc4a346a-b5e5-49cd-a056-f600d538a65f", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "076e6cbb-8e89-40b0-87a1-95f307b3bd7e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2017-05-14T17:47:38-07:00", + "end": "2018-05-14T17:47:38-07:00" + }, + "created": "2017-05-14T17:47:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:076e6cbb-8e89-40b0-87a1-95f307b3bd7e" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 224.83200000000002, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98", + "resource": { + "resourceType": "Encounter", + "id": "bbeaf98d-63d5-4ade-be01-1f622ead5a98", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:23:38-07:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:23:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:223ecd8e-0fb0-4a36-af9d-30209d7e8087", + "resource": { + "resourceType": "Procedure", + "id": "223ecd8e-0fb0-4a36-af9d-30209d7e8087", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" + }, + "performedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:08:38-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ebde62e3-4086-4e07-bbb1-24f532835804", + "resource": { + "resourceType": "Claim", + "id": "ebde62e3-4086-4e07-bbb1-24f532835804", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:23:38-07:00" + }, + "created": "2017-05-14T18:23:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:223ecd8e-0fb0-4a36-af9d-30209d7e8087" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "encounter": [ + { + "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" + } + ] + }, + { + "sequence": 2, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "net": { + "value": 18980.64, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5b200d19-06de-42b6-85ce-eac88e5aaa45", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5b200d19-06de-42b6-85ce-eac88e5aaa45", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ebde62e3-4086-4e07-bbb1-24f532835804" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2017-05-14T18:23:38-07:00", + "end": "2018-05-14T18:23:38-07:00" + }, + "created": "2017-05-14T18:23:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:ebde62e3-4086-4e07-bbb1-24f532835804" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + }, + "servicedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:23:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "73761001", + "display": "Colonoscopy" + } + ], + "text": "Colonoscopy" + }, + "servicedPeriod": { + "start": "2017-05-14T17:32:38-07:00", + "end": "2017-05-14T18:23:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 18980.64, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 3796.128, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 15184.512, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 18980.64, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 18980.64, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 15184.512, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29", + "resource": { + "resourceType": "Encounter", + "id": "c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:48468fdb-4817-42ff-9b70-06794275b9a8", + "resource": { + "resourceType": "Observation", + "id": "48468fdb-4817-42ff-9b70-06794275b9a8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3738c5c3-2754-44a3-8316-39068d145e91", + "resource": { + "resourceType": "Observation", + "id": "3738c5c3-2754-44a3-8316-39068d145e91", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:88483de3-63c0-48e7-b3b4-5a84b4b2d4cb", + "resource": { + "resourceType": "Observation", + "id": "88483de3-63c0-48e7-b3b4-5a84b4b2d4cb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 82.7, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5dc7c11-8e61-4cb6-bf4c-abe89f1743ab", + "resource": { + "resourceType": "Observation", + "id": "a5dc7c11-8e61-4cb6-bf4c-abe89f1743ab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 28.51, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6081858c-f14a-4780-aabd-5797c532fd5a", + "resource": { + "resourceType": "Observation", + "id": "6081858c-f14a-4780-aabd-5797c532fd5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 120, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cba93d27-bf57-4085-86dc-3f9814913545", + "resource": { + "resourceType": "Observation", + "id": "cba93d27-bf57-4085-86dc-3f9814913545", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 66, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9a36f421-3850-4e11-8565-2cc9001bb3dc", + "resource": { + "resourceType": "Observation", + "id": "9a36f421-3850-4e11-8565-2cc9001bb3dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c315d2e8-6337-4a41-9623-f590369795d0", + "resource": { + "resourceType": "Observation", + "id": "c315d2e8-6337-4a41-9623-f590369795d0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "effectiveDateTime": "2018-05-20T17:32:38-07:00", + "issued": "2018-05-20T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:81abebcd-0ab9-4d20-8a5f-c7726af47076", + "resource": { + "resourceType": "Procedure", + "id": "81abebcd-0ab9-4d20-8a5f-c7726af47076", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "performedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T17:47:38-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ad206db7-3a4e-4243-9964-e9a011e7e599", + "resource": { + "resourceType": "Immunization", + "id": "ad206db7-3a4e-4243-9964-e9a011e7e599", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "occurrenceDateTime": "2018-05-20T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:70783cd0-a45e-4d39-abef-cdb681275231", + "resource": { + "resourceType": "Immunization", + "id": "70783cd0-a45e-4d39-abef-cdb681275231", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "occurrenceDateTime": "2018-05-20T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7bb1566f-b6c8-4528-8fb6-fdcdd490ba48", + "resource": { + "resourceType": "Immunization", + "id": "7bb1566f-b6c8-4528-8fb6-fdcdd490ba48", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + }, + "occurrenceDateTime": "2018-05-20T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b173915f-7894-4b30-b6ed-2babd254dc71", + "resource": { + "resourceType": "Claim", + "id": "b173915f-7894-4b30-b6ed-2babd254dc71", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "created": "2018-05-20T18:02:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ad206db7-3a4e-4243-9964-e9a011e7e599" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:70783cd0-a45e-4d39-abef-cdb681275231" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7bb1566f-b6c8-4528-8fb6-fdcdd490ba48" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:81abebcd-0ab9-4d20-8a5f-c7726af47076" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + }, + { + "sequence": 5, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "net": { + "value": 665.14, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f41e5d5d-cc37-42fe-b983-b22cca1855e3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f41e5d5d-cc37-42fe-b983-b22cca1855e3", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b173915f-7894-4b30-b6ed-2babd254dc71" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2018-05-20T18:02:38-07:00", + "end": "2019-05-20T18:02:38-07:00" + }, + "created": "2018-05-20T18:02:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b173915f-7894-4b30-b6ed-2babd254dc71" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "121", + "display": "zoster" + } + ], + "text": "zoster" + }, + "servicedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationSequence": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 4, + "informationSequence": [ + 3 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "servicedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "servicedPeriod": { + "start": "2018-05-20T17:32:38-07:00", + "end": "2018-05-20T18:02:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 665.14, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 133.028, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 532.112, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 665.14, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 665.14, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 869.36, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04", + "resource": { + "resourceType": "Encounter", + "id": "f3d3fd85-ab22-43fb-8c38-d25621657a04", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2019-05-26T17:32:38-07:00", + "end": "2019-05-26T17:47:38-07:00" + }, + "individual": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "display": "Dr. Doretta917 Goldner995" + } + } + ], + "period": { + "start": "2019-05-26T17:32:38-07:00", + "end": "2019-05-26T17:47:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cb5a50cc-b8f6-4a5b-b2c1-ae3504d37bea", + "resource": { + "resourceType": "Observation", + "id": "cb5a50cc-b8f6-4a5b-b2c1-ae3504d37bea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:96b0ce29-9b6e-47c0-906c-21993090b3cc", + "resource": { + "resourceType": "Observation", + "id": "96b0ce29-9b6e-47c0-906c-21993090b3cc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8da3621a-3caa-4949-8306-77c6df70f0b8", + "resource": { + "resourceType": "Observation", + "id": "8da3621a-3caa-4949-8306-77c6df70f0b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 81.2, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3fdaed1f-a5ad-490f-8ba9-42f1f97fbbcc", + "resource": { + "resourceType": "Observation", + "id": "3fdaed1f-a5ad-490f-8ba9-42f1f97fbbcc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 28, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fdfd4246-dacb-4452-b200-2e4c43d4bd28", + "resource": { + "resourceType": "Observation", + "id": "fdfd4246-dacb-4452-b200-2e4c43d4bd28", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 72, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 102, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:87f9f1bb-f715-43bc-a731-15e167f0a205", + "resource": { + "resourceType": "Observation", + "id": "87f9f1bb-f715-43bc-a731-15e167f0a205", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34f92871-ad58-40d2-98c5-a54529678d69", + "resource": { + "resourceType": "Observation", + "id": "34f92871-ad58-40d2-98c5-a54529678d69", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a95619d-84a9-41ea-aca9-ec18f5b74f73", + "resource": { + "resourceType": "Observation", + "id": "8a95619d-84a9-41ea-aca9-ec18f5b74f73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 8.663, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4754fea8-3cde-40d2-9e69-0e960a45b01f", + "resource": { + "resourceType": "Observation", + "id": "4754fea8-3cde-40d2-9e69-0e960a45b01f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 4.1903, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbf04538-bb3f-4ad6-abd9-d71470327d11", + "resource": { + "resourceType": "Observation", + "id": "bbf04538-bb3f-4ad6-abd9-d71470327d11", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 14.815, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6ea04164-003b-4a17-9912-8a1dc3da6851", + "resource": { + "resourceType": "Observation", + "id": "6ea04164-003b-4a17-9912-8a1dc3da6851", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 49.421, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b5c8fa29-9df2-4800-b84e-bead29dc9cff", + "resource": { + "resourceType": "Observation", + "id": "b5c8fa29-9df2-4800-b84e-bead29dc9cff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 82.277, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6edb1ce-7e50-44bc-9557-9f390bc97da5", + "resource": { + "resourceType": "Observation", + "id": "c6edb1ce-7e50-44bc-9557-9f390bc97da5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 30.975, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0acf358c-881e-43d5-8e1f-8ff090e31723", + "resource": { + "resourceType": "Observation", + "id": "0acf358c-881e-43d5-8e1f-8ff090e31723", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 33.195, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12b0aabe-4522-474b-a0d2-6ec95ce0b10d", + "resource": { + "resourceType": "Observation", + "id": "12b0aabe-4522-474b-a0d2-6ec95ce0b10d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 41.104, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1425323-297b-46b9-90d6-98a36e912771", + "resource": { + "resourceType": "Observation", + "id": "d1425323-297b-46b9-90d6-98a36e912771", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 233.63, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b080b4fa-ce5c-4187-b36e-6f517dfe44e8", + "resource": { + "resourceType": "Observation", + "id": "b080b4fa-ce5c-4187-b36e-6f517dfe44e8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 167.11, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a99738d3-4ff9-4154-b055-2e8b204fb676", + "resource": { + "resourceType": "Observation", + "id": "a99738d3-4ff9-4154-b055-2e8b204fb676", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueQuantity": { + "value": 11.285, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2bfaa421-4b59-45ea-924a-e41c5268378e", + "resource": { + "resourceType": "Observation", + "id": "2bfaa421-4b59-45ea-924a-e41c5268378e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eb5f18a0-e47c-468b-ba6b-6321b8bd6b02", + "resource": { + "resourceType": "Immunization", + "id": "eb5f18a0-e47c-468b-ba6b-6321b8bd6b02", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "occurrenceDateTime": "2019-05-26T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:24291062-f162-49bc-8b23-c6483ecf3dac", + "resource": { + "resourceType": "DiagnosticReport", + "id": "24291062-f162-49bc-8b23-c6483ecf3dac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v2-0074", + "code": "LAB", + "display": "Laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + }, + "effectiveDateTime": "2019-05-26T17:32:38-07:00", + "issued": "2019-05-26T17:32:38.789-07:00", + "result": [ + { + "reference": "urn:uuid:8a95619d-84a9-41ea-aca9-ec18f5b74f73", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:4754fea8-3cde-40d2-9e69-0e960a45b01f", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:bbf04538-bb3f-4ad6-abd9-d71470327d11", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:6ea04164-003b-4a17-9912-8a1dc3da6851", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:b5c8fa29-9df2-4800-b84e-bead29dc9cff", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:c6edb1ce-7e50-44bc-9557-9f390bc97da5", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:0acf358c-881e-43d5-8e1f-8ff090e31723", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:12b0aabe-4522-474b-a0d2-6ec95ce0b10d", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:d1425323-297b-46b9-90d6-98a36e912771", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b080b4fa-ce5c-4187-b36e-6f517dfe44e8", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:a99738d3-4ff9-4154-b055-2e8b204fb676", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7cdd17aa-39d6-4423-add1-22b9bf6e0dab", + "resource": { + "resourceType": "Claim", + "id": "7cdd17aa-39d6-4423-add1-22b9bf6e0dab", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2019-05-26T17:32:38-07:00", + "end": "2019-05-26T17:47:38-07:00" + }, + "created": "2019-05-26T17:47:38-07:00", + "provider": { + "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "display": "PCP83180" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:eb5f18a0-e47c-468b-ba6b-6321b8bd6b02" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5774b0f5-269e-4d39-8a5b-8aae6726f0e4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5774b0f5-269e-4d39-8a5b-8aae6726f0e4", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "performer": [ + { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7cdd17aa-39d6-4423-add1-22b9bf6e0dab" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2019-05-26T17:47:38-07:00", + "end": "2020-05-26T17:47:38-07:00" + }, + "created": "2019-05-26T17:47:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:7cdd17aa-39d6-4423-add1-22b9bf6e0dab" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + }, + "servicedPeriod": { + "start": "2019-05-26T17:32:38-07:00", + "end": "2019-05-26T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2019-05-26T17:32:38-07:00", + "end": "2019-05-26T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "19", + "display": "Off Campus-Outpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1", + "resource": { + "resourceType": "Encounter", + "id": "4969b899-5b8e-4f3b-8196-6b8a2315aeb1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", + "resource": { + "resourceType": "Condition", + "id": "cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", + "clinicalStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", + "code": "resolved" + } + ] + }, + "verificationStatus": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", + "code": "confirmed" + } + ] + }, + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + }, + "onsetDateTime": "2020-03-13T17:32:38-07:00", + "abatementDateTime": "2020-03-27T17:32:38-07:00", + "recordedDate": "2020-03-13T17:32:38-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:de0b0703-7f5a-4e1d-8ad3-eb085a00801a", + "resource": { + "resourceType": "Procedure", + "id": "de0b0703-7f5a-4e1d-8ad3-eb085a00801a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + }, + "performedPeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T17:45:38-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", + "display": "Acute bronchitis (disorder)" + }, + { + "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", + "display": "Acute bronchitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9", + "resource": { + "resourceType": "MedicationRequest", + "id": "c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + }, + "authoredOn": "2020-03-13T17:32:38-07:00", + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + }, + "reasonReference": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + }, + { + "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:06e84f61-f998-48c5-8aeb-dc5dca4786be", + "resource": { + "resourceType": "Claim", + "id": "06e84f61-f998-48c5-8aeb-dc5dca4786be", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "pharmacy" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "created": "2020-03-13T18:00:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "prescription": { + "reference": "urn:uuid:c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9" + }, + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + } + ] + } + ], + "total": { + "value": 7.85, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:025332c4-e014-4146-bf51-47ebcdb8bfdc", + "resource": { + "resourceType": "CareTeam", + "id": "025332c4-e014-4146-bf51-47ebcdb8bfdc", + "status": "active", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + }, + "period": { + "start": "2020-03-13T17:32:38-07:00" + }, + "participant": [ + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "116153009", + "display": "Patient" + } + ], + "text": "Patient" + } + ], + "member": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "223366009", + "display": "Healthcare professional (occupation)" + } + ], + "text": "Healthcare professional (occupation)" + } + ], + "member": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + }, + { + "role": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "224891009", + "display": "Healthcare services (qualifier value)" + } + ], + "text": "Healthcare services (qualifier value)" + } + ], + "member": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + } + ], + "reasonCode": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + ], + "managingOrganization": [ + { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + ] + }, + "request": { + "method": "POST", + "url": "CareTeam" + } + }, + { + "fullUrl": "urn:uuid:042d6c6e-8e0d-46b1-ada1-c9cbe45f959f", + "resource": { + "resourceType": "CarePlan", + "id": "042d6c6e-8e0d-46b1-ada1-c9cbe45f959f", + "text": { + "status": "generated", + "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " + }, + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + }, + "period": { + "start": "2020-03-13T17:32:38-07:00" + }, + "careTeam": [ + { + "reference": "urn:uuid:025332c4-e014-4146-bf51-47ebcdb8bfdc" + } + ], + "addresses": [ + { + "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" + }, + { + "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "in-progress", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "in-progress", + "location": { + "display": "MOUNT AUBURN HOSPITAL" + } + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:25f7923e-88e7-4249-8e74-b0f07da9e00c", + "resource": { + "resourceType": "Claim", + "id": "25f7923e-88e7-4249-8e74-b0f07da9e00c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "created": "2020-03-13T18:00:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:de0b0703-7f5a-4e1d-8ad3-eb085a00801a" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "encounter": [ + { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + } + }, + { + "sequence": 3, + "procedureSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "net": { + "value": 7186.11, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7178d1d7-f7c6-458e-a7a6-212eb707fbbb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7178d1d7-f7c6-458e-a7a6-212eb707fbbb", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "Humana" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "Humana" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "25f7923e-88e7-4249-8e74-b0f07da9e00c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2020-03-13T18:00:38-07:00", + "end": "2021-03-13T18:00:38-08:00" + }, + "created": "2020-03-13T18:00:38-07:00", + "insurer": { + "display": "Humana" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:25f7923e-88e7-4249-8e74-b0f07da9e00c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" + }, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "Humana" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + }, + "servicedPeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" + } + ] + }, + { + "sequence": 2, + "diagnosisSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "servicedPeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "servicedPeriod": { + "start": "2020-03-13T17:32:38-07:00", + "end": "2020-03-13T18:00:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 7186.11, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1437.222, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 5748.888, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 7186.11, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 7186.11, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 5748.888, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0", + "resource": { + "resourceType": "Encounter", + "id": "59e9b993-6536-4a29-94b4-06b0ac1c47d0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Mr. Seymour882 Shanahan202" + }, + "participant": [ + { + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", + "code": "PPRF", + "display": "primary performer" + } + ], + "text": "primary performer" + } + ], + "period": { + "start": "2020-03-22T17:32:38-07:00", + "end": "2020-03-22T17:47:38-07:00" + }, + "individual": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "display": "Dr. Ja391 Streich926" + } + } + ], + "period": { + "start": "2020-03-22T17:32:38-07:00", + "end": "2020-03-22T17:47:38-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2f0a548b-44dd-46d6-bd8e-c335c6f7598d", + "resource": { + "resourceType": "Observation", + "id": "2f0a548b-44dd-46d6-bd8e-c335c6f7598d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 170.3, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aa117c1a-2047-4635-b705-09096602ac2c", + "resource": { + "resourceType": "Observation", + "id": "aa117c1a-2047-4635-b705-09096602ac2c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ef26cfb-f9d7-4fb5-b827-a36c4e4cf244", + "resource": { + "resourceType": "Observation", + "id": "5ef26cfb-f9d7-4fb5-b827-a36c4e4cf244", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 82.9, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0fa9f9af-3fb7-4f3f-82b8-abe91edbc464", + "resource": { + "resourceType": "Observation", + "id": "0fa9f9af-3fb7-4f3f-82b8-abe91edbc464", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 28.59, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b89f5128-bafe-4f0e-830b-5ca840622f5b", + "resource": { + "resourceType": "Observation", + "id": "b89f5128-bafe-4f0e-830b-5ca840622f5b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:73d21f72-87f9-43ce-a273-ec51c563d673", + "resource": { + "resourceType": "Observation", + "id": "73d21f72-87f9-43ce-a273-ec51c563d673", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b696278a-7117-4c92-96db-a217891fa3bb", + "resource": { + "resourceType": "Observation", + "id": "b696278a-7117-4c92-96db-a217891fa3bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fdc15f45-8944-40c8-ac25-b68398ea4e01", + "resource": { + "resourceType": "Observation", + "id": "fdc15f45-8944-40c8-ac25-b68398ea4e01", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "effectiveDateTime": "2020-03-22T17:32:38-07:00", + "issued": "2020-03-22T17:32:38.789-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:46fab8e3-cde4-4dcf-a0e5-d814380ce826", + "resource": { + "resourceType": "Immunization", + "id": "46fab8e3-cde4-4dcf-a0e5-d814380ce826", + "status": "completed", + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "encounter": { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + }, + "occurrenceDateTime": "2020-03-22T17:32:38-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b8b07ad6-eae4-4e45-b82b-a30d4836203c", + "resource": { + "resourceType": "Claim", + "id": "b8b07ad6-eae4-4e45-b82b-a30d4836203c", + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", + "display": "Seymour882 Shanahan202" + }, + "billablePeriod": { + "start": "2020-03-22T17:32:38-07:00", + "end": "2020-03-22T17:47:38-07:00" + }, + "created": "2020-03-22T17:47:38-07:00", + "provider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "display": "MOUNT AUBURN HOSPITAL" + }, + "priority": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/processpriority", + "code": "normal" + } + ] + }, + "supportingInfo": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:46fab8e3-cde4-4dcf-a0e5-d814380ce826" + } + } + ], + "insurance": [ + { + "sequence": 1, + "focal": true, + "coverage": { + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "encounter": [ + { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "net": { + "value": 140.52, + "currency": "USD" + } + } + ], + "total": { + "value": 129.16, + "currency": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d11b0238-af92-47c0-b1c7-a1fee525262c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d11b0238-af92-47c0-b1c7-a1fee525262c", + "contained": [ + { + "resourceType": "ServiceRequest", + "id": "referral", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "requester": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "performer": [ + { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "status": "active", + "type": { + "text": "NO_INSURANCE" + }, + "beneficiary": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "payor": [ + { + "display": "NO_INSURANCE" + } + ] + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b8b07ad6-eae4-4e45-b82b-a30d4836203c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claim-type", + "code": "institutional" + } + ] + }, + "use": "claim", + "patient": { + "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" + }, + "billablePeriod": { + "start": "2020-03-22T17:47:38-07:00", + "end": "2021-03-22T17:47:38-07:00" + }, + "created": "2020-03-22T17:47:38-07:00", + "insurer": { + "display": "NO_INSURANCE" + }, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "referral": { + "reference": "#referral" + }, + "claim": { + "reference": "urn:uuid:b8b07ad6-eae4-4e45-b82b-a30d4836203c" + }, + "outcome": "complete", + "careTeam": [ + { + "sequence": 1, + "provider": { + "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" + }, + "role": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": [ + { + "focal": true, + "coverage": { + "reference": "#coverage", + "display": "NO_INSURANCE" + } + } + ], + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + }, + "servicedPeriod": { + "start": "2020-03-22T17:32:38-07:00", + "end": "2020-03-22T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" + } + ] + }, + { + "sequence": 2, + "informationSequence": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "productOrService": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "servicedPeriod": { + "start": "2020-03-22T17:32:38-07:00", + "end": "2020-03-22T17:47:38-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "currency": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "currency": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "total": [ + { + "category": { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/adjudication", + "code": "submitted", + "display": "Submitted Amount" + } + ], + "text": "Submitted Amount" + }, + "amount": { + "value": 129.16, + "currency": "USD" + } + } + ], + "payment": { + "amount": { + "value": 112.41600000000001, + "currency": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json new file mode 100644 index 000000000000..0584f07f2676 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json @@ -0,0 +1,1182 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 20 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "resource": { + "resourceType": "Organization", + "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 4 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAMBRIDGE HEALTH ALLIANCE", + "telecom": [ + { + "system": "phone", + "value": "6176652300" + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", + "resource": { + "resourceType": "Organization", + "id": "ef6ab57c-ed94-3dbe-9861-812d515918b3", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 199 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ef6ab57c-ed94-3dbe-9861-812d515918b3" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAPE COD HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5087711800" + } + ], + "address": [ + { + "line": [ + "88 LEWIS BAY ROAD" + ], + "city": "HYANNIS", + "state": "MA", + "postalCode": "02601", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", + "resource": { + "resourceType": "Organization", + "id": "b0e04623-b02c-3f8b-92ea-943fc4db60da", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 38 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "b0e04623-b02c-3f8b-92ea-943fc4db60da" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "LOWELL GENERAL HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "9789376000" + } + ], + "address": [ + { + "line": [ + "295 VARNUM AVENUE" + ], + "city": "LOWELL", + "state": "MA", + "postalCode": "01854", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d78e84ec-30aa-3bba-a33a-f29a3a454662", + "resource": { + "resourceType": "Organization", + "id": "d78e84ec-30aa-3bba-a33a-f29a3a454662", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 10 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d78e84ec-30aa-3bba-a33a-f29a3a454662" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MASSACHUSETTS GENERAL HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6177262000" + } + ], + "address": [ + { + "line": [ + "55 FRUIT STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", + "resource": { + "resourceType": "Organization", + "id": "23834663-ed53-3da9-b330-d6e1ecb8428e", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 12 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "23834663-ed53-3da9-b330-d6e1ecb8428e" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SOUTHCOAST HOSPITAL GROUP, INC", + "telecom": [ + { + "system": "phone", + "value": "5086793131" + } + ], + "address": [ + { + "line": [ + "363 HIGHLAND AVENUE" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02720", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", + "resource": { + "resourceType": "Organization", + "id": "44bef9d3-91c2-3005-93e0-ccf436348ff0", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 29 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "44bef9d3-91c2-3005-93e0-ccf436348ff0" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MASSACHUSETTS EYE AND EAR INFIRMARY -", + "telecom": [ + { + "system": "phone", + "value": "6175237900" + } + ], + "address": [ + { + "line": [ + "243 CHARLES STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", + "resource": { + "resourceType": "Organization", + "id": "d733d4a9-080d-3593-b910-2366e652b7ea", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 79 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "d733d4a9-080d-3593-b910-2366e652b7ea" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6179837000" + } + ], + "address": [ + { + "line": [ + "1153 CENTRE STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02130", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:ff9863d3-3fa3-3861-900e-f00148f5d9c2", + "resource": { + "resourceType": "Organization", + "id": "ff9863d3-3fa3-3861-900e-f00148f5d9c2", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "ff9863d3-3fa3-3861-900e-f00148f5d9c2" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SHRINERS' HOSPITAL FOR CHILDREN - BOSTON, THE", + "telecom": [ + { + "system": "phone", + "value": "6177223000" + } + ], + "address": [ + { + "line": [ + "51 BLOSSOM STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "resource": { + "resourceType": "Organization", + "id": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 42 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 38 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 30 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 15 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC", + "telecom": [ + { + "system": "phone", + "value": "978-667-8600" + } + ], + "address": [ + { + "line": [ + "2 ANDOVER RD" + ], + "city": "BILLERICA", + "state": "MA", + "postalCode": "01821-1916", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "resource": { + "resourceType": "Organization", + "id": "7f7c08c8-1c39-3b26-bd34-5122a003dd91", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 42 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 29 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 16 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 18 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "7f7c08c8-1c39-3b26-bd34-5122a003dd91" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP83180", + "telecom": [ + { + "system": "phone", + "value": "617-389-6951" + } + ], + "address": [ + { + "line": [ + "107 FERRY ST" + ], + "city": "EVERETT", + "state": "MA", + "postalCode": "02149-4940", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "resource": { + "resourceType": "Organization", + "id": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 81 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 515 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 85 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 104 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP128586", + "address": [ + { + "line": [ + "2360 CRANBERRY HWY" + ], + "city": "W WAREHAM", + "state": "MA", + "postalCode": "02576-1208", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "resource": { + "resourceType": "Organization", + "id": "75bdb2ec-cbd0-35af-b0fe-18b111890b67", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 25 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 15 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 4 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 8 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "75bdb2ec-cbd0-35af-b0fe-18b111890b67" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP235727", + "address": [ + { + "line": [ + "67 SLADES FERRY BLVD" + ], + "city": "SOMERSET", + "state": "MA", + "postalCode": "02726", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "resource": { + "resourceType": "Organization", + "id": "76a80350-916e-3fc9-8fed-2c8c4b36c42f", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 54 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 321 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 48 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 55 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "76a80350-916e-3fc9-8fed-2c8c4b36c42f" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC.", + "telecom": [ + { + "system": "phone", + "value": "617-912-7900" + } + ], + "address": [ + { + "line": [ + "301 BROADWAY" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "02150-2807", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "resource": { + "resourceType": "Organization", + "id": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "TRU MEDICAL WALK IN CENTER", + "telecom": [ + { + "system": "phone", + "value": "508-675-1522" + } + ], + "address": [ + { + "line": [ + "528 NEWTON STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "2721", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:7116c200-6663-3503-8314-1ee16845e5d3", + "resource": { + "resourceType": "Organization", + "id": "7116c200-6663-3503-8314-1ee16845e5d3", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "7116c200-6663-3503-8314-1ee16845e5d3" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SAINTS WALK-IN MEDICAL CENTER - URGENT CARE AND OCCUPATIONAL HEALTH", + "telecom": [ + { + "system": "phone", + "value": "978-458-6868" + } + ], + "address": [ + { + "line": [ + "85 PARKHURST ROAD" + ], + "city": "CHELMSFORD", + "state": "MA", + "postalCode": "1824", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "resource": { + "resourceType": "Organization", + "id": "3265f387-6c51-32ee-8f6d-b2a89caa34d5", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "3265f387-6c51-32ee-8f6d-b2a89caa34d5" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN", + "telecom": [ + { + "system": "phone", + "value": "617-884-8300" + } + ], + "address": [ + { + "line": [ + "151 EVERETT AVENUE" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "2150", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:0f7f97f6-693c-3de9-8190-85feaf4632e2", + "resource": { + "resourceType": "Organization", + "id": "0f7f97f6-693c-3de9-8190-85feaf4632e2", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "0f7f97f6-693c-3de9-8190-85feaf4632e2" + } + ], + "active": true, + "type": [ + { + "coding": [ + { + "system": "http://terminology.hl7.org/CodeSystem/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CONCENTRA URGENT CARE - EAST BOSTON", + "telecom": [ + { + "system": "phone", + "value": "617-568-6500" + } + ], + "address": [ + { + "line": [ + "1 HARBORSIDE DRIVE" + ], + "city": "EAST BOSTON", + "state": "MA", + "postalCode": "2128", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json new file mode 100644 index 000000000000..c2d6b7250129 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json @@ -0,0 +1,978 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", + "resource": { + "resourceType": "Practitioner", + "id": "eabb178d-5fde-3f2a-b2b0-84b601205578", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 20 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999989" + } + ], + "active": true, + "name": [ + { + "family": "Streich926", + "given": [ + "Ja391" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Ja391.Streich926@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "resource": { + "resourceType": "Practitioner", + "id": "e4e8b8e3-604e-3bca-978b-afcfa63491d0", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 4 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999959" + } + ], + "active": true, + "name": [ + { + "family": "Harvey63", + "given": [ + "Alla648" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Alla648.Harvey63@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", + "resource": { + "resourceType": "Practitioner", + "id": "fb3b949e-e3cb-3069-b280-521ac0512d2e", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 199 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999949" + } + ], + "active": true, + "name": [ + { + "family": "Kautzer186", + "given": [ + "Heath320" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Heath320.Kautzer186@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "88 LEWIS BAY ROAD" + ], + "city": "HYANNIS", + "state": "MA", + "postalCode": "02601", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "resource": { + "resourceType": "Practitioner", + "id": "57adca03-a2e1-3753-bcc5-e6ef86bbb821", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 38 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999759" + } + ], + "active": true, + "name": [ + { + "family": "Tillman293", + "given": [ + "Franklin857" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Franklin857.Tillman293@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "295 VARNUM AVENUE" + ], + "city": "LOWELL", + "state": "MA", + "postalCode": "01854", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:7b0a96ca-9d87-3146-96e7-ef4e67128176", + "resource": { + "resourceType": "Practitioner", + "id": "7b0a96ca-9d87-3146-96e7-ef4e67128176", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 10 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999719" + } + ], + "active": true, + "name": [ + { + "family": "Oberbrunner298", + "given": [ + "Herlinda751" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Herlinda751.Oberbrunner298@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "55 FRUIT STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", + "resource": { + "resourceType": "Practitioner", + "id": "f411d1de-1d0c-365f-8a71-8d225cb962af", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 12 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999699" + } + ], + "active": true, + "name": [ + { + "family": "Brakus656", + "given": [ + "Gillian484" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Gillian484.Brakus656@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "363 HIGHLAND AVENUE" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "02720", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "resource": { + "resourceType": "Practitioner", + "id": "fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 29 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999689" + } + ], + "active": true, + "name": [ + { + "family": "Mayert710", + "given": [ + "Dovie983" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Dovie983.Mayert710@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "243 CHARLES STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", + "resource": { + "resourceType": "Practitioner", + "id": "3a576225-3a87-3704-a672-e1c8bc03389f", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 79 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999519" + } + ], + "active": true, + "name": [ + { + "family": "Howell947", + "given": [ + "Max124" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Max124.Howell947@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "1153 CENTRE STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02130", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:4c2aa782-3c72-3d1c-be80-70a3bae80c38", + "resource": { + "resourceType": "Practitioner", + "id": "4c2aa782-3c72-3d1c-be80-70a3bae80c38", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999999379" + } + ], + "active": true, + "name": [ + { + "family": "Graham902", + "given": [ + "Spencer878" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Spencer878.Graham902@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "51 BLOSSOM STREET" + ], + "city": "BOSTON", + "state": "MA", + "postalCode": "02114", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", + "resource": { + "resourceType": "Practitioner", + "id": "0d5ab589-1d26-346a-b8db-e425d2e139e7", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 42 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999969899" + } + ], + "active": true, + "name": [ + { + "family": "Gerlach374", + "given": [ + "Lillia547" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Lillia547.Gerlach374@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "2 ANDOVER RD" + ], + "city": "BILLERICA", + "state": "MA", + "postalCode": "01821-1916", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", + "resource": { + "resourceType": "Practitioner", + "id": "12810b32-78af-32f0-b85c-8177e3f2e2e2", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 42 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999959759" + } + ], + "active": true, + "name": [ + { + "family": "Goldner995", + "given": [ + "Doretta917" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Doretta917.Goldner995@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "107 FERRY ST" + ], + "city": "EVERETT", + "state": "MA", + "postalCode": "02149-4940", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", + "resource": { + "resourceType": "Practitioner", + "id": "cf99bf43-51e5-3bab-8623-211cd48f66dc", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 81 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999951739" + } + ], + "active": true, + "name": [ + { + "family": "DuBuque211", + "given": [ + "Carlyn477" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Carlyn477.DuBuque211@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "2360 CRANBERRY HWY" + ], + "city": "W WAREHAM", + "state": "MA", + "postalCode": "02576-1208", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "resource": { + "resourceType": "Practitioner", + "id": "b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 25 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999931859" + } + ], + "active": true, + "name": [ + { + "family": "Mante251", + "given": [ + "Bertie593" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Bertie593.Mante251@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "67 SLADES FERRY BLVD" + ], + "city": "SOMERSET", + "state": "MA", + "postalCode": "02726", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", + "resource": { + "resourceType": "Practitioner", + "id": "0390078a-1894-335b-9136-5f518ee1dbb6", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 54 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999931799" + } + ], + "active": true, + "name": [ + { + "family": "Koelpin146", + "given": [ + "Cheryle584" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Cheryle584.Koelpin146@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "301 BROADWAY" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "02150-2807", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", + "resource": { + "resourceType": "Practitioner", + "id": "21a54030-cce5-3104-a773-9eb9bd60830b", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906799" + } + ], + "active": true, + "name": [ + { + "family": "Eichmann909", + "given": [ + "Wanetta360" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Wanetta360.Eichmann909@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "528 NEWTON STREET" + ], + "city": "FALL RIVER", + "state": "MA", + "postalCode": "2721", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:389e216f-1825-347e-b23b-630d2f37b195", + "resource": { + "resourceType": "Practitioner", + "id": "389e216f-1825-347e-b23b-630d2f37b195", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906719" + } + ], + "active": true, + "name": [ + { + "family": "Watsica258", + "given": [ + "Gidget756" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Gidget756.Watsica258@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "85 PARKHURST ROAD" + ], + "city": "CHELMSFORD", + "state": "MA", + "postalCode": "1824", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", + "resource": { + "resourceType": "Practitioner", + "id": "fe82a096-86b9-344b-b405-bd24b6ad4715", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906709" + } + ], + "active": true, + "name": [ + { + "family": "Hoppe518", + "given": [ + "Devin82" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Devin82.Hoppe518@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "151 EVERETT AVENUE" + ], + "city": "CHELSEA", + "state": "MA", + "postalCode": "2150", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:1296d3c9-02e0-34a2-86e1-0e185a8f92d6", + "resource": { + "resourceType": "Practitioner", + "id": "1296d3c9-02e0-34a2-86e1-0e185a8f92d6", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "9999906459" + } + ], + "active": true, + "name": [ + { + "family": "Ankunding277", + "given": [ + "Elton404" + ], + "prefix": [ + "Dr." + ] + } + ], + "telecom": [ + { + "system": "email", + "value": "Elton404.Ankunding277@example.com", + "use": "work" + } + ], + "address": [ + { + "line": [ + "1 HARBORSIDE DRIVE" + ], + "city": "EAST BOSTON", + "state": "MA", + "postalCode": "2128", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json new file mode 100644 index 000000000000..986c874a86e7 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json @@ -0,0 +1,16003 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796", + "resource": { + "resourceType": "Patient", + "id": "02a63c07-9fcc-42ba-aec0-9d5399ac4796", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 4798728652809701405 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2054-5", + "display": "Black or African American" + } + }, + { + "url": "text", + "valueString": "Black or African American" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Mina319 Langworth352" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "M" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Winchester", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.0805149386188746 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 0.9194850613811254 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-16-7591" + } + ], + "name": [ + { + "use": "official", + "family": "Romaguera67", + "given": [ + "Alexander630" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-213-6346", + "use": "home" + } + ], + "gender": "male", + "birthDate": "2018-07-02", + "deceasedDateTime": "2020-04-13T06:13:10-07:00", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.35034978194894 + }, + { + "url": "longitude", + "valueDecimal": -71.11270110210596 + } + ] + } + ], + "line": [ + "639 Rogahn Grove" + ], + "city": "Cambridge", + "state": "Massachusetts", + "postalCode": "02138", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5", + "resource": { + "resourceType": "Organization", + "id": "aced4ce4-957a-3d86-b20a-302d12f949a5", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "LONGFELLOW PRIMARY CARE, PC", + "telecom": [ + { + "system": "phone", + "value": "617-492-4545" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.376043 + }, + { + "url": "longitude", + "valueDecimal": -71.11868 + } + ] + } + ], + "line": [ + "625 MOUNT AUBURN ST" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138-4518", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886", + "resource": { + "resourceType": "Practitioner", + "id": "29e6e246-9052-309c-a47f-c2e5bc60c886", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "24070" + } + ], + "active": true, + "name": [ + { + "family": "Waelchi213", + "given": [ + "Cristopher265" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "625 MOUNT AUBURN ST" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138-4518", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492", + "resource": { + "resourceType": "Encounter", + "id": "7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fd976131-d3fc-459a-a942-2cbde9b4ded2", + "resource": { + "resourceType": "Immunization", + "id": "fd976131-d3fc-459a-a942-2cbde9b4ded2", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492" + }, + "date": "2018-07-02T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ac6048bf-c392-489e-a72c-fb2b13c1c181", + "resource": { + "resourceType": "Claim", + "id": "ac6048bf-c392-489e-a72c-fb2b13c1c181", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:fd976131-d3fc-459a-a942-2cbde9b4ded2" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:750f8437-b065-4c2b-bb4b-05b8b1a90ed8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "750f8437-b065-4c2b-bb4b-05b8b1a90ed8", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ac6048bf-c392-489e-a72c-fb2b13c1c181" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-07-02T06:28:10-07:00", + "end": "2019-07-02T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", + "resource": { + "resourceType": "Practitioner", + "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10" + } + ], + "active": true, + "name": [ + { + "family": "Jenkins714", + "given": [ + "Lara964" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47", + "resource": { + "resourceType": "Encounter", + "id": "b0454769-6ade-48ac-90f1-2ab20e998a47", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency Room Admission" + } + ], + "text": "Emergency Room Admission" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "128613002", + "display": "Seizure disorder" + } + ] + } + ], + "hospitalization": { + "dischargeDisposition": { + "coding": [ + { + "system": "http://www.nubc.org/patient-discharge", + "code": "01", + "display": "Discharged to home care or self care (routine discharge)" + } + ], + "text": "Discharged to home care or self care (routine discharge)" + } + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92", + "resource": { + "resourceType": "Condition", + "id": "317bc0bf-3dc2-4233-8b6f-3540d1bc9e92", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "128613002", + "display": "Seizure disorder" + } + ], + "text": "Seizure disorder" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "onsetDateTime": "2018-07-02T06:13:10-07:00", + "assertedDate": "2018-07-02T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113", + "resource": { + "resourceType": "Condition", + "id": "8c6f2229-a6ab-4397-ace4-b4ecaeb78113", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "703151001", + "display": "History of single seizure (situation)" + } + ], + "text": "History of single seizure (situation)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "onsetDateTime": "2018-07-02T06:13:10-07:00", + "assertedDate": "2018-07-02T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:449ade9a-34c5-4e70-a39b-df8ad1f3f87c", + "resource": { + "resourceType": "Observation", + "id": "449ade9a-34c5-4e70-a39b-df8ad1f3f87c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 52.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84d0c337-8875-4c8c-adff-06590ae484d8", + "resource": { + "resourceType": "Observation", + "id": "84d0c337-8875-4c8c-adff-06590ae484d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e1cdff5b-fd65-45b9-9371-64fd2eba01ed", + "resource": { + "resourceType": "Observation", + "id": "e1cdff5b-fd65-45b9-9371-64fd2eba01ed", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 3.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49baefdd-a2f6-4af6-b819-6a9ecbceb309", + "resource": { + "resourceType": "Observation", + "id": "49baefdd-a2f6-4af6-b819-6a9ecbceb309", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.11714, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:800e4f80-0a10-4b4c-84a4-4287fec2b32a", + "resource": { + "resourceType": "Observation", + "id": "800e4f80-0a10-4b4c-84a4-4287fec2b32a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 35.720, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:acd19516-ac3e-48f8-92dd-3eddfdce847e", + "resource": { + "resourceType": "Observation", + "id": "acd19516-ac3e-48f8-92dd-3eddfdce847e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:187c0917-46dd-4849-b48f-b7736c147be4", + "resource": { + "resourceType": "Observation", + "id": "187c0917-46dd-4849-b48f-b7736c147be4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 80, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd47a3de-5003-4dda-bf22-d436d8bb34de", + "resource": { + "resourceType": "Observation", + "id": "dd47a3de-5003-4dda-bf22-d436d8bb34de", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bc594c64-c12c-4901-b534-efd4cc16812a", + "resource": { + "resourceType": "Observation", + "id": "bc594c64-c12c-4901-b534-efd4cc16812a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 8.1663, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:507829cc-ba3d-481b-8739-481dbb4971da", + "resource": { + "resourceType": "Observation", + "id": "507829cc-ba3d-481b-8739-481dbb4971da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 5.2914, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7b8b65d-2008-4e69-b4ad-761096d58385", + "resource": { + "resourceType": "Observation", + "id": "b7b8b65d-2008-4e69-b4ad-761096d58385", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 15.174, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a92e98ac-976f-4f7c-af03-ef6684e9f7e9", + "resource": { + "resourceType": "Observation", + "id": "a92e98ac-976f-4f7c-af03-ef6684e9f7e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 38.942, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", + "resource": { + "resourceType": "Observation", + "id": "ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 83.049, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", + "resource": { + "resourceType": "Observation", + "id": "b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 28.367, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f0944211-bbcd-46b9-9349-0a3e754b1bc7", + "resource": { + "resourceType": "Observation", + "id": "f0944211-bbcd-46b9-9349-0a3e754b1bc7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 35.103, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:74983c05-e0b0-4b59-9a17-735d84341aa2", + "resource": { + "resourceType": "Observation", + "id": "74983c05-e0b0-4b59-9a17-735d84341aa2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 43.812, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cdfef0be-8691-4652-aaa9-0cbeee650aa9", + "resource": { + "resourceType": "Observation", + "id": "cdfef0be-8691-4652-aaa9-0cbeee650aa9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 243.61, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7910aa61-9b39-4967-8a11-c70d76cbc1c1", + "resource": { + "resourceType": "Observation", + "id": "7910aa61-9b39-4967-8a11-c70d76cbc1c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 153.31, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4a4355e0-07b2-4193-960b-3d26f391d60d", + "resource": { + "resourceType": "Observation", + "id": "4a4355e0-07b2-4193-960b-3d26f391d60d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 9.8262, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e377311f-78bd-44ef-8963-8c3c8209e629", + "resource": { + "resourceType": "Observation", + "id": "e377311f-78bd-44ef-8963-8c3c8209e629", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddff5e15-83b7-484a-91fa-be6e84f0df22", + "resource": { + "resourceType": "Procedure", + "id": "ddff5e15-83b7-484a-91fa-be6e84f0df22", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "54550000", + "display": "Seizure Count Cerebral Cortex Electroencephalogram (EEG)" + } + ], + "text": "Seizure Count Cerebral Cortex Electroencephalogram (EEG)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "performedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T08:50:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:82160634-a224-42b2-b4e4-77cf9129d853", + "resource": { + "resourceType": "MedicationRequest", + "id": "82160634-a224-42b2-b4e4-77cf9129d853", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308971", + "display": "Carbamazepine[Tegretol]" + } + ], + "text": "Carbamazepine[Tegretol]" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "authoredOn": "2018-07-02T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:c7498b56-a644-480d-88dc-7d190761879b", + "resource": { + "resourceType": "Claim", + "id": "c7498b56-a644-480d-88dc-7d190761879b", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:82160634-a224-42b2-b4e4-77cf9129d853" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + } + ] + } + ], + "total": { + "value": 325.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d05cec1c-89e2-456f-9839-f39af8419f28", + "resource": { + "resourceType": "DiagnosticReport", + "id": "d05cec1c-89e2-456f-9839-f39af8419f28", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "result": [ + { + "reference": "urn:uuid:bc594c64-c12c-4901-b534-efd4cc16812a", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:507829cc-ba3d-481b-8739-481dbb4971da", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b7b8b65d-2008-4e69-b4ad-761096d58385", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:a92e98ac-976f-4f7c-af03-ef6684e9f7e9", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:f0944211-bbcd-46b9-9349-0a3e754b1bc7", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:74983c05-e0b0-4b59-9a17-735d84341aa2", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:cdfef0be-8691-4652-aaa9-0cbeee650aa9", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:7910aa61-9b39-4967-8a11-c70d76cbc1c1", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:4a4355e0-07b2-4193-960b-3d26f391d60d", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f051abc6-d9d8-40bc-97be-e534b8b475c2", + "resource": { + "resourceType": "Claim", + "id": "f051abc6-d9d8-40bc-97be-e534b8b475c2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92" + } + }, + { + "sequence": 2, + "diagnosisReference": { + "reference": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ddff5e15-83b7-484a-91fa-be6e84f0df22" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "diagnosisLinkId": [ + 2 + ] + }, + { + "sequence": 4, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "54550000" + } + ] + }, + "net": { + "value": 12986.76, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ea48dada-6e46-4b3b-9ce9-bcd08b2e47f5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ea48dada-6e46-4b3b-9ce9-bcd08b2e47f5", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f051abc6-d9d8-40bc-97be-e534b8b475c2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-07-02T09:50:10-07:00", + "end": "2019-07-02T09:50:10-07:00" + }, + "created": "2018-07-02T09:50:10-07:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:f051abc6-d9d8-40bc-97be-e534b8b475c2" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 2, + "diagnosisReference": { + "reference": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + }, + { + "sequence": 3, + "diagnosisLinkId": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "54550000" + } + ] + }, + "servicedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T09:50:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "net": { + "value": 12986.76, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2597.3520000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 10389.408000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 12986.76, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 12986.76, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 10389.408000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12", + "resource": { + "resourceType": "Encounter", + "id": "4488ce28-6761-4741-9c26-3d8d8648af12", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2018-08-06T06:13:10-07:00", + "end": "2018-08-06T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bafc638d-077c-49e8-b900-859bdf3d6fc7", + "resource": { + "resourceType": "Observation", + "id": "bafc638d-077c-49e8-b900-859bdf3d6fc7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 56.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fec7ca8f-cbfa-4d06-bea6-093a161de5c7", + "resource": { + "resourceType": "Observation", + "id": "fec7ca8f-cbfa-4d06-bea6-093a161de5c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3cf3e132-dcfa-481e-9857-603b6ab58483", + "resource": { + "resourceType": "Observation", + "id": "3cf3e132-dcfa-481e-9857-603b6ab58483", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 4, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:971bc36f-971a-463c-bbb4-30baec5c6b13", + "resource": { + "resourceType": "Observation", + "id": "971bc36f-971a-463c-bbb4-30baec5c6b13", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.018783, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70a25cb8-f762-4b97-aaf7-04e5cdf7fe78", + "resource": { + "resourceType": "Observation", + "id": "70a25cb8-f762-4b97-aaf7-04e5cdf7fe78", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 39.130, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5ad6b974-9707-4de9-8103-41faf1e22cf0", + "resource": { + "resourceType": "Observation", + "id": "5ad6b974-9707-4de9-8103-41faf1e22cf0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 72, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e3e25ae-e1a6-43dd-875c-bb7176bb250b", + "resource": { + "resourceType": "Observation", + "id": "0e3e25ae-e1a6-43dd-875c-bb7176bb250b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 77, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6e4cc34-6827-48e2-8dbd-70679f8a0a1f", + "resource": { + "resourceType": "Observation", + "id": "e6e4cc34-6827-48e2-8dbd-70679f8a0a1f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c430effa-3a2c-497c-ad11-0f9ad5d4fb19", + "resource": { + "resourceType": "Observation", + "id": "c430effa-3a2c-497c-ad11-0f9ad5d4fb19", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:07b0a7c4-a136-4b04-b462-f62807d223c8", + "resource": { + "resourceType": "Procedure", + "id": "07b0a7c4-a136-4b04-b462-f62807d223c8", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "performedPeriod": { + "start": "2018-08-06T06:13:10-07:00", + "end": "2018-08-06T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ab69207f-d574-4100-84f0-174f94becc16", + "resource": { + "resourceType": "Immunization", + "id": "ab69207f-d574-4100-84f0-174f94becc16", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + }, + "date": "2018-08-06T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1b9403c4-b51b-4650-861d-948b9710a45a", + "resource": { + "resourceType": "Claim", + "id": "1b9403c4-b51b-4650-861d-948b9710a45a", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-08-06T06:13:10-07:00", + "end": "2018-08-06T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ab69207f-d574-4100-84f0-174f94becc16" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:07b0a7c4-a136-4b04-b462-f62807d223c8" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 541.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6f1c7897-af99-495d-b271-8f796fff068b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6f1c7897-af99-495d-b271-8f796fff068b", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1b9403c4-b51b-4650-861d-948b9710a45a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-08-06T06:43:10-07:00", + "end": "2019-08-06T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 108.24400000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 432.97600000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 541.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 541.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 545.392, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b", + "resource": { + "resourceType": "Encounter", + "id": "49fa8b6c-d685-441d-b10d-02ec4c92bf2b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2018-10-08T06:13:10-07:00", + "end": "2018-10-08T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5e95ade0-7a20-483b-9774-788107e3005b", + "resource": { + "resourceType": "Observation", + "id": "5e95ade0-7a20-483b-9774-788107e3005b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 62, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f26c36eb-a7aa-47a0-92e3-687f204e3b4b", + "resource": { + "resourceType": "Observation", + "id": "f26c36eb-a7aa-47a0-92e3-687f204e3b4b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:734525b0-1e22-4f51-89aa-13ca80261d11", + "resource": { + "resourceType": "Observation", + "id": "734525b0-1e22-4f51-89aa-13ca80261d11", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 5.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b4a017b1-4e7f-4901-9c13-2f56e14c64dc", + "resource": { + "resourceType": "Observation", + "id": "b4a017b1-4e7f-4901-9c13-2f56e14c64dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.42442, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:74c76fe8-f12b-4c0b-9667-427af9bf647a", + "resource": { + "resourceType": "Observation", + "id": "74c76fe8-f12b-4c0b-9667-427af9bf647a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 41.690, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d729006-4be6-4bd5-a279-7966b37d1ba5", + "resource": { + "resourceType": "Observation", + "id": "7d729006-4be6-4bd5-a279-7966b37d1ba5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d2d0467-cede-4ace-af92-861e79ff6f48", + "resource": { + "resourceType": "Observation", + "id": "2d2d0467-cede-4ace-af92-861e79ff6f48", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 67, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7ccf38c6-c989-4cea-9ea0-92715ed4860c", + "resource": { + "resourceType": "Observation", + "id": "7ccf38c6-c989-4cea-9ea0-92715ed4860c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b648e60e-bf23-46b4-b9c9-3a7a393e80ac", + "resource": { + "resourceType": "Observation", + "id": "b648e60e-bf23-46b4-b9c9-3a7a393e80ac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e04b895-97d8-4d94-a3af-3e55ad057f4f", + "resource": { + "resourceType": "Immunization", + "id": "4e04b895-97d8-4d94-a3af-3e55ad057f4f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc", + "resource": { + "resourceType": "Immunization", + "id": "fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:eabfbeb5-e820-480a-80a7-535f53e0c3b5", + "resource": { + "resourceType": "Immunization", + "id": "eabfbeb5-e820-480a-80a7-535f53e0c3b5", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a9682836-9b4b-4d94-b5e0-433fc52b6eab", + "resource": { + "resourceType": "Immunization", + "id": "a9682836-9b4b-4d94-b5e0-433fc52b6eab", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8a624fd6-4c8f-446a-a021-c0f8fee0d15b", + "resource": { + "resourceType": "Immunization", + "id": "8a624fd6-4c8f-446a-a021-c0f8fee0d15b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0f99487b-0475-43ef-9869-1fe7591c1d55", + "resource": { + "resourceType": "Claim", + "id": "0f99487b-0475-43ef-9869-1fe7591c1d55", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-10-08T06:13:10-07:00", + "end": "2018-10-08T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4e04b895-97d8-4d94-a3af-3e55ad057f4f" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:eabfbeb5-e820-480a-80a7-535f53e0c3b5" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a9682836-9b4b-4d94-b5e0-433fc52b6eab" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8a624fd6-4c8f-446a-a021-c0f8fee0d15b" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d4960c1d-5052-4012-85dd-9aa154b3238d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d4960c1d-5052-4012-85dd-9aa154b3238d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0f99487b-0475-43ef-9869-1fe7591c1d55" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-10-08T06:28:10-07:00", + "end": "2019-10-08T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d", + "resource": { + "resourceType": "Encounter", + "id": "58c3e07d-b23e-4d7c-a85e-9662620a062d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:43:10-08:00" + }, + "hospitalization": { + "dischargeDisposition": { + "coding": [ + { + "system": "http://www.nubc.org/patient-discharge", + "code": "01", + "display": "Discharged to home care or self care (routine discharge)" + } + ], + "text": "Discharged to home care or self care (routine discharge)" + } + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:442de3a6-9b3e-47aa-a449-228f2b259d22", + "resource": { + "resourceType": "Observation", + "id": "442de3a6-9b3e-47aa-a449-228f2b259d22", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 66, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d811f25-d09a-491e-9671-3677b1439524", + "resource": { + "resourceType": "Observation", + "id": "7d811f25-d09a-491e-9671-3677b1439524", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c546661-cfcb-4e91-bb0f-0b4ec8124acf", + "resource": { + "resourceType": "Observation", + "id": "7c546661-cfcb-4e91-bb0f-0b4ec8124acf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 6.6000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee5b8473-cb89-4e0e-abe5-94bef351621b", + "resource": { + "resourceType": "Observation", + "id": "ee5b8473-cb89-4e0e-abe5-94bef351621b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 3.2575, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b124abeb-b1e6-410c-ae5b-e07ada3dc650", + "resource": { + "resourceType": "Observation", + "id": "b124abeb-b1e6-410c-ae5b-e07ada3dc650", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 43.340, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:21f96e6d-9db4-4bb5-b5ca-6c70aa155358", + "resource": { + "resourceType": "Observation", + "id": "21f96e6d-9db4-4bb5-b5ca-6c70aa155358", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2e36292e-9eb2-42ab-a81d-a3c737e16fab", + "resource": { + "resourceType": "Observation", + "id": "2e36292e-9eb2-42ab-a81d-a3c737e16fab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4fbdd32a-79ab-48f0-9ac2-f728539af7bc", + "resource": { + "resourceType": "Observation", + "id": "4fbdd32a-79ab-48f0-9ac2-f728539af7bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6dde0d43-ca5b-4a01-b528-ee24df4ddf53", + "resource": { + "resourceType": "Observation", + "id": "6dde0d43-ca5b-4a01-b528-ee24df4ddf53", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecde24ef-7159-4ad5-9bdf-df56d857f7cb", + "resource": { + "resourceType": "Procedure", + "id": "ecde24ef-7159-4ad5-9bdf-df56d857f7cb", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "performedPeriod": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:28:10-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e645deb5-749a-471e-8964-0d5a3f340a20", + "resource": { + "resourceType": "MedicationRequest", + "id": "e645deb5-749a-471e-8964-0d5a3f340a20", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308971", + "display": "Carbamazepine[Tegretol]" + } + ], + "text": "Carbamazepine[Tegretol]" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "authoredOn": "2018-12-10T05:13:10-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + }, + "onBehalfOf": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:cd306f6b-9691-4503-afe1-90dc0fee38b2", + "resource": { + "resourceType": "Claim", + "id": "cd306f6b-9691-4503-afe1-90dc0fee38b2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:43:10-08:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "prescription": { + "reference": "urn:uuid:e645deb5-749a-471e-8964-0d5a3f340a20" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + } + ] + } + ], + "total": { + "value": 109.78, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:935d5e93-e972-4d27-8ab7-38afe2a51f91", + "resource": { + "resourceType": "Immunization", + "id": "935d5e93-e972-4d27-8ab7-38afe2a51f91", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:35f80ad5-66d2-49d0-a5fe-68653071b0de", + "resource": { + "resourceType": "Immunization", + "id": "35f80ad5-66d2-49d0-a5fe-68653071b0de", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4d5ae690-6eaf-41c2-bee1-8b2681b26ef8", + "resource": { + "resourceType": "Immunization", + "id": "4d5ae690-6eaf-41c2-bee1-8b2681b26ef8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:259c0a73-dc9e-41ee-b821-ce1aadb2faff", + "resource": { + "resourceType": "Immunization", + "id": "259c0a73-dc9e-41ee-b821-ce1aadb2faff", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7b669084-0084-4c89-b97d-59fc357cfa79", + "resource": { + "resourceType": "Immunization", + "id": "7b669084-0084-4c89-b97d-59fc357cfa79", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:72cb0040-8ca0-4c6d-8f30-263006270063", + "resource": { + "resourceType": "Claim", + "id": "72cb0040-8ca0-4c6d-8f30-263006270063", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:43:10-08:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:935d5e93-e972-4d27-8ab7-38afe2a51f91" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:35f80ad5-66d2-49d0-a5fe-68653071b0de" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4d5ae690-6eaf-41c2-bee1-8b2681b26ef8" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:259c0a73-dc9e-41ee-b821-ce1aadb2faff" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7b669084-0084-4c89-b97d-59fc357cfa79" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:ecde24ef-7159-4ad5-9bdf-df56d857f7cb" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 306.13, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:94b41851-7237-43ec-a540-ef01cc05503d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "94b41851-7237-43ec-a540-ef01cc05503d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "72cb0040-8ca0-4c6d-8f30-263006270063" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-12-10T05:43:10-08:00", + "end": "2019-12-10T05:43:10-08:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 61.226, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 244.904, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 306.13, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 306.13, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 806.984, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762", + "resource": { + "resourceType": "Encounter", + "id": "7f9b3142-62ee-4763-a46c-f733c123a762", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2019-03-11T06:13:10-07:00", + "end": "2019-03-11T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c8521971-2409-470d-b629-78b2e562907a", + "resource": { + "resourceType": "Observation", + "id": "c8521971-2409-470d-b629-78b2e562907a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 70.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b78488df-4f15-44d7-85f7-9cc577ad6114", + "resource": { + "resourceType": "Observation", + "id": "b78488df-4f15-44d7-85f7-9cc577ad6114", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:224c5de2-616a-4aeb-af70-92849e48bedf", + "resource": { + "resourceType": "Observation", + "id": "224c5de2-616a-4aeb-af70-92849e48bedf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 7.9000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1755540b-e461-4aa3-8994-cd6d2b3af948", + "resource": { + "resourceType": "Observation", + "id": "1755540b-e461-4aa3-8994-cd6d2b3af948", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 7.3337, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f398044e-2bb4-4387-85fe-97397b05cb76", + "resource": { + "resourceType": "Observation", + "id": "f398044e-2bb4-4387-85fe-97397b05cb76", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 44.990, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4f9f6c3-e7f5-4b67-8015-d0519a4a1ace", + "resource": { + "resourceType": "Observation", + "id": "c4f9f6c3-e7f5-4b67-8015-d0519a4a1ace", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 107, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5fbb7b6-2a73-4729-88e1-cf49312d947d", + "resource": { + "resourceType": "Observation", + "id": "a5fbb7b6-2a73-4729-88e1-cf49312d947d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 73, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71c86983-4255-4f27-83f8-c8d74bfbc56e", + "resource": { + "resourceType": "Observation", + "id": "71c86983-4255-4f27-83f8-c8d74bfbc56e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1bbb0806-655b-4936-bf63-e32f3b4d6646", + "resource": { + "resourceType": "Observation", + "id": "1bbb0806-655b-4936-bf63-e32f3b4d6646", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:af277a3c-5cc6-42a9-8548-5425320b7193", + "resource": { + "resourceType": "Procedure", + "id": "af277a3c-5cc6-42a9-8548-5425320b7193", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "performedPeriod": { + "start": "2019-03-11T06:13:10-07:00", + "end": "2019-03-11T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:323b5d1e-e933-46d1-b613-60455686a0b3", + "resource": { + "resourceType": "Immunization", + "id": "323b5d1e-e933-46d1-b613-60455686a0b3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e24b017a-362f-45d4-be05-ab94edf20ac9", + "resource": { + "resourceType": "Immunization", + "id": "e24b017a-362f-45d4-be05-ab94edf20ac9", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c7051b58-38f9-41de-924a-f177c87bfe0c", + "resource": { + "resourceType": "Immunization", + "id": "c7051b58-38f9-41de-924a-f177c87bfe0c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0abe138d-5c05-46a9-8922-15fa27b0ae4e", + "resource": { + "resourceType": "Immunization", + "id": "0abe138d-5c05-46a9-8922-15fa27b0ae4e", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:496449ca-cd40-4cbc-a07b-ebdb28832c92", + "resource": { + "resourceType": "Immunization", + "id": "496449ca-cd40-4cbc-a07b-ebdb28832c92", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b007e486-5c0a-402e-a7da-e7234215dd64", + "resource": { + "resourceType": "Claim", + "id": "b007e486-5c0a-402e-a7da-e7234215dd64", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-03-11T06:13:10-07:00", + "end": "2019-03-11T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:323b5d1e-e933-46d1-b613-60455686a0b3" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e24b017a-362f-45d4-be05-ab94edf20ac9" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c7051b58-38f9-41de-924a-f177c87bfe0c" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:0abe138d-5c05-46a9-8922-15fa27b0ae4e" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:496449ca-cd40-4cbc-a07b-ebdb28832c92" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:af277a3c-5cc6-42a9-8548-5425320b7193" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 455.84, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:903d5a6b-46f2-4886-b290-642c13d1a2c8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "903d5a6b-46f2-4886-b290-642c13d1a2c8", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b007e486-5c0a-402e-a7da-e7234215dd64" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-03-11T06:43:10-07:00", + "end": "2020-03-11T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 91.168, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 364.672, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 455.84, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 455.84, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 926.7520000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba", + "resource": { + "resourceType": "Encounter", + "id": "7e1e68c2-c941-411d-9008-1edd35390bba", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-05-31T06:13:10-07:00", + "end": "2019-05-31T06:43:10-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47", + "resource": { + "resourceType": "Condition", + "id": "41b800ee-0377-47d3-a656-7cf5c49f8a47", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" + }, + "onsetDateTime": "2019-05-31T06:13:10-07:00", + "abatementDateTime": "2019-06-09T06:13:10-07:00", + "assertedDate": "2019-05-31T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f054dc09-7aee-485e-a07e-a49f45b70a11", + "resource": { + "resourceType": "Observation", + "id": "f054dc09-7aee-485e-a07e-a49f45b70a11", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" + }, + "effectiveDateTime": "2019-05-31T06:13:10-07:00", + "issued": "2019-05-31T06:13:10.107-07:00", + "valueQuantity": { + "value": 37.204, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:86ae819c-bff4-428a-a62d-ce767110ea6d", + "resource": { + "resourceType": "Procedure", + "id": "86ae819c-bff4-428a-a62d-ce767110ea6d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" + }, + "performedPeriod": { + "start": "2019-05-31T06:13:10-07:00", + "end": "2019-05-31T06:28:10-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47", + "display": "Acute viral pharyngitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b7c9497e-c1db-4f15-bdf2-2cf78163b154", + "resource": { + "resourceType": "Claim", + "id": "b7c9497e-c1db-4f15-bdf2-2cf78163b154", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-05-31T06:13:10-07:00", + "end": "2019-05-31T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:86ae819c-bff4-428a-a62d-ce767110ea6d" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "net": { + "value": 2006.36, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cd691ae2-9136-4be9-9a88-6b436f5f1b10", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cd691ae2-9136-4be9-9a88-6b436f5f1b10", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b7c9497e-c1db-4f15-bdf2-2cf78163b154" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-05-31T06:43:10-07:00", + "end": "2020-05-31T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 401.272, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1605.088, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2006.36, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2006.36, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1605.088, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843", + "resource": { + "resourceType": "Encounter", + "id": "b7b6f2ee-41e5-4010-912d-f63e6f2b7843", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2019-06-10T06:13:10-07:00", + "end": "2019-06-10T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d4690ecb-a5e4-472b-a773-26ffef7cd6d6", + "resource": { + "resourceType": "Observation", + "id": "d4690ecb-a5e4-472b-a773-26ffef7cd6d6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 74.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:764ceafa-2dff-4361-8a8d-95b3e3bc59cd", + "resource": { + "resourceType": "Observation", + "id": "764ceafa-2dff-4361-8a8d-95b3e3bc59cd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:741be9b9-87cf-43c7-a069-4e0a4dd06651", + "resource": { + "resourceType": "Observation", + "id": "741be9b9-87cf-43c7-a069-4e0a4dd06651", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 8.8000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf87b938-858c-42d3-835b-8d28c90b7717", + "resource": { + "resourceType": "Observation", + "id": "cf87b938-858c-42d3-835b-8d28c90b7717", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 8.3146, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66ff3c55-6b91-4695-8219-43af239d6711", + "resource": { + "resourceType": "Observation", + "id": "66ff3c55-6b91-4695-8219-43af239d6711", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 46.130, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:64c391ce-23b9-4257-83e0-cd6133d52079", + "resource": { + "resourceType": "Observation", + "id": "64c391ce-23b9-4257-83e0-cd6133d52079", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 120, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:37d2185b-720b-40bb-a96e-a1c25783dd81", + "resource": { + "resourceType": "Observation", + "id": "37d2185b-720b-40bb-a96e-a1c25783dd81", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:757efdd1-f513-463e-a960-db0cd95c411a", + "resource": { + "resourceType": "Observation", + "id": "757efdd1-f513-463e-a960-db0cd95c411a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b0b3116-c1fd-4a5f-9af9-e543a07391b8", + "resource": { + "resourceType": "Observation", + "id": "4b0b3116-c1fd-4a5f-9af9-e543a07391b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c0103e5-f76a-4d85-b0f0-18afd7c82df1", + "resource": { + "resourceType": "Claim", + "id": "5c0103e5-f76a-4d85-b0f0-18afd7c82df1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-06-10T06:13:10-07:00", + "end": "2019-06-10T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:04f81bb1-b318-4a02-abf8-0203ab4d2992", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "04f81bb1-b318-4a02-abf8-0203ab4d2992", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5c0103e5-f76a-4d85-b0f0-18afd7c82df1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-06-10T06:28:10-07:00", + "end": "2020-06-10T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2", + "resource": { + "resourceType": "Encounter", + "id": "2071123a-e5b2-44e7-b27d-67b03437e6b2", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T07:13:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d", + "resource": { + "resourceType": "Condition", + "id": "9e7ec09d-4469-46e7-b90c-279a291a582d", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "44465007", + "display": "Sprain of ankle" + } + ], + "text": "Sprain of ankle" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + }, + "onsetDateTime": "2019-07-27T06:13:10-07:00", + "abatementDateTime": "2019-08-24T06:13:10-07:00", + "assertedDate": "2019-07-27T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:7aac826d-7e6d-487f-85f5-2ad1d864ffe0", + "resource": { + "resourceType": "MedicationRequest", + "id": "7aac826d-7e6d-487f-85f5-2ad1d864ffe0", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + }, + "authoredOn": "2019-07-27T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1e69cc56-a03f-465a-8cf3-dd662dc91f37", + "resource": { + "resourceType": "Claim", + "id": "1e69cc56-a03f-465a-8cf3-dd662dc91f37", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T07:13:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:7aac826d-7e6d-487f-85f5-2ad1d864ffe0" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + } + ] + } + ], + "total": { + "value": 4.93, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6613c2da-3c5b-4499-a6c4-18a47ff1aa4b", + "resource": { + "resourceType": "CarePlan", + "id": "6613c2da-3c5b-4499-a6c4-18a47ff1aa4b", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "91251008", + "display": "Physical therapy procedure" + } + ], + "text": "Physical therapy procedure" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + }, + "period": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-08-24T06:13:10-07:00" + }, + "addresses": [ + { + "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229586001", + "display": "Rest, ice, compression and elevation treatment programme" + } + ], + "text": "Rest, ice, compression and elevation treatment programme" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "229070002", + "display": "Stretching exercises" + } + ], + "text": "Stretching exercises" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:fbdf870f-6856-4759-b821-d721ee3fe9fd", + "resource": { + "resourceType": "Claim", + "id": "fbdf870f-6856-4759-b821-d721ee3fe9fd", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T07:13:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:27678b31-0de7-492c-9574-e0866c6eea6f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "27678b31-0de7-492c-9574-e0866c6eea6f", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fbdf870f-6856-4759-b821-d721ee3fe9fd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-07-27T07:13:10-07:00", + "end": "2020-07-27T07:13:10-07:00" + }, + "created": "2019-07-27T07:13:10-07:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:fbdf870f-6856-4759-b821-d721ee3fe9fd" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T07:13:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T07:13:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf", + "resource": { + "resourceType": "Encounter", + "id": "cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f2474743-e217-4fce-9348-ab283d81d698", + "resource": { + "resourceType": "CarePlan", + "id": "f2474743-e217-4fce-9348-ab283d81d698", + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "384758001", + "display": "Self-care interventions (procedure)" + } + ], + "text": "Self-care interventions (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf" + }, + "period": { + "start": "2019-07-27T06:13:10-07:00" + }, + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "409002", + "display": "Food allergy diet" + } + ], + "text": "Food allergy diet" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58332002", + "display": "Allergy education" + } + ], + "text": "Allergy education" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58332002", + "display": "Allergy education" + } + ], + "text": "Allergy education" + }, + "status": "in-progress" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:67bbb47b-b627-411b-a12e-e8b8b4073fa0", + "resource": { + "resourceType": "Claim", + "id": "67bbb47b-b627-411b-a12e-e8b8b4073fa0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-07-27T06:13:10-07:00", + "end": "2019-07-27T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d7509e59-6617-434a-b10c-7a2792dc2031", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d7509e59-6617-434a-b10c-7a2792dc2031", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "67bbb47b-b627-411b-a12e-e8b8b4073fa0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-07-27T06:28:10-07:00", + "end": "2020-07-27T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862", + "resource": { + "resourceType": "Encounter", + "id": "6427742c-bd2c-4f4c-aea4-644f5db83862", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-08-11T06:13:10-07:00", + "end": "2019-08-11T07:06:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6db8de76-eae5-4950-8a42-2607004386fd", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "6db8de76-eae5-4950-8a42-2607004386fd", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "232347008", + "display": "Dander (animal) allergy" + } + ], + "text": "Dander (animal) allergy" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "assertedDate": "2019-08-11T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:e35c9f21-9093-4e84-9b36-615ec2d1c210", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "e35c9f21-9093-4e84-9b36-615ec2d1c210", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "300913006", + "display": "Shellfish allergy" + } + ], + "text": "Shellfish allergy" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "assertedDate": "2019-08-11T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:625e1745-69bc-426b-aca1-10e83724f849", + "resource": { + "resourceType": "Observation", + "id": "625e1745-69bc-426b-aca1-10e83724f849", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6206-7", + "display": "Peanut IgE Ab in Serum" + } + ], + "text": "Peanut IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.10165, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ca6457f-b5f8-4203-8ad5-be9b1c140026", + "resource": { + "resourceType": "Observation", + "id": "0ca6457f-b5f8-4203-8ad5-be9b1c140026", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6273-7", + "display": "Walnut IgE Ab in Serum" + } + ], + "text": "Walnut IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.19633, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:de18b9d6-84e1-40d3-ab38-0b97a85537c5", + "resource": { + "resourceType": "Observation", + "id": "de18b9d6-84e1-40d3-ab38-0b97a85537c5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6082-2", + "display": "Codfish IgE Ab in Serum" + } + ], + "text": "Codfish IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.14530, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:07d68b6c-2868-4402-95cc-8ad376c3d8e1", + "resource": { + "resourceType": "Observation", + "id": "07d68b6c-2868-4402-95cc-8ad376c3d8e1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6246-3", + "display": "Shrimp IgE Ab in Serum" + } + ], + "text": "Shrimp IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 12.952, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40530c35-df47-4333-84b4-ee421dd431b8", + "resource": { + "resourceType": "Observation", + "id": "40530c35-df47-4333-84b4-ee421dd431b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6276-0", + "display": "Wheat IgE Ab in Serum" + } + ], + "text": "Wheat IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.34990, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77c14a36-4b3c-49b5-824d-9079efd3e215", + "resource": { + "resourceType": "Observation", + "id": "77c14a36-4b3c-49b5-824d-9079efd3e215", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6106-9", + "display": "Egg white IgE Ab in Serum" + } + ], + "text": "Egg white IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.31980, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5d40e233-c707-4989-94f2-a734f60e878d", + "resource": { + "resourceType": "Observation", + "id": "5d40e233-c707-4989-94f2-a734f60e878d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6248-9", + "display": "Soybean IgE Ab in Serum" + } + ], + "text": "Soybean IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.12029, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:85ca8f32-f13f-4ab6-80dd-a198fdc07d5d", + "resource": { + "resourceType": "Observation", + "id": "85ca8f32-f13f-4ab6-80dd-a198fdc07d5d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "7258-7", + "display": "Cow milk IgE Ab in Serum" + } + ], + "text": "Cow milk IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.34789, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c421021b-5667-40ec-af43-0f77d7429d24", + "resource": { + "resourceType": "Observation", + "id": "c421021b-5667-40ec-af43-0f77d7429d24", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6189-5", + "display": "White oak IgE Ab in Serum" + } + ], + "text": "White oak IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.27479, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40ae2b98-265a-4626-9b5e-b3cbaf728fa4", + "resource": { + "resourceType": "Observation", + "id": "40ae2b98-265a-4626-9b5e-b3cbaf728fa4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6085-5", + "display": "Common Ragweed IgE Ab in Serum" + } + ], + "text": "Common Ragweed IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.18551, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a87c4cc7-bda8-4c88-a570-0fc40e10fdf9", + "resource": { + "resourceType": "Observation", + "id": "a87c4cc7-bda8-4c88-a570-0fc40e10fdf9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6833-8", + "display": "Cat dander IgE Ab in Serum" + } + ], + "text": "Cat dander IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 24.424, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a45da77-5881-4753-b85e-836bd3161449", + "resource": { + "resourceType": "Observation", + "id": "3a45da77-5881-4753-b85e-836bd3161449", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6095-4", + "display": "American house dust mite IgE Ab in Serum" + } + ], + "text": "American house dust mite IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.20303, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:688791a4-03f4-45ab-8462-2c41b22cfc3c", + "resource": { + "resourceType": "Observation", + "id": "688791a4-03f4-45ab-8462-2c41b22cfc3c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6075-6", + "display": "Cladosporium herbarum IgE Ab in Serum" + } + ], + "text": "Cladosporium herbarum IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.23219, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5de5308-150c-4eef-a9de-d76e06f02656", + "resource": { + "resourceType": "Observation", + "id": "f5de5308-150c-4eef-a9de-d76e06f02656", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6844-5", + "display": "Honey bee IgE Ab in Serum" + } + ], + "text": "Honey bee IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.13334, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f898ef50-0e14-4344-8c54-57ca6e5208c7", + "resource": { + "resourceType": "Observation", + "id": "f898ef50-0e14-4344-8c54-57ca6e5208c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6158-0", + "display": "Latex IgE Ab in Serum" + } + ], + "text": "Latex IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "effectiveDateTime": "2019-08-11T06:13:10-07:00", + "issued": "2019-08-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.31837, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e309a89d-4eef-4383-b200-901f2b6a60e5", + "resource": { + "resourceType": "Procedure", + "id": "e309a89d-4eef-4383-b200-901f2b6a60e5", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "395142003", + "display": "Allergy screening test" + } + ], + "text": "Allergy screening test" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "performedPeriod": { + "start": "2019-08-11T06:13:10-07:00", + "end": "2019-08-11T06:51:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e2256164-5e0e-49fe-bd55-ae3b268b7901", + "resource": { + "resourceType": "MedicationRequest", + "id": "e2256164-5e0e-49fe-bd55-ae3b268b7901", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1014676", + "display": "cetirizine hydrochloride 5 MG Oral Tablet" + } + ], + "text": "cetirizine hydrochloride 5 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "authoredOn": "2019-08-11T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:75b5cd63-28a6-468e-9fdd-ebbdbe88987e", + "resource": { + "resourceType": "Claim", + "id": "75b5cd63-28a6-468e-9fdd-ebbdbe88987e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-08-11T06:13:10-07:00", + "end": "2019-08-11T07:06:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:e2256164-5e0e-49fe-bd55-ae3b268b7901" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + } + ] + } + ], + "total": { + "value": 20.99, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b5db7fae-fb16-47e5-8eb4-39ace8c18c17", + "resource": { + "resourceType": "MedicationRequest", + "id": "b5db7fae-fb16-47e5-8eb4-39ace8c18c17", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1870230", + "display": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" + } + ], + "text": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + }, + "authoredOn": "2019-08-11T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8b72f548-4002-453b-b0bc-aa8caeda7b49", + "resource": { + "resourceType": "Claim", + "id": "8b72f548-4002-453b-b0bc-aa8caeda7b49", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-08-11T06:13:10-07:00", + "end": "2019-08-11T07:06:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:b5db7fae-fb16-47e5-8eb4-39ace8c18c17" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + } + ] + } + ], + "total": { + "value": 258.81, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8b74a937-8421-4f63-b9a2-423e182e62b3", + "resource": { + "resourceType": "Claim", + "id": "8b74a937-8421-4f63-b9a2-423e182e62b3", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-08-11T06:13:10-07:00", + "end": "2019-08-11T07:06:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e309a89d-4eef-4383-b200-901f2b6a60e5" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "395142003" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:da38110f-ae59-40ac-ad2b-bc65ba12bf11", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "da38110f-ae59-40ac-ad2b-bc65ba12bf11", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8b74a937-8421-4f63-b9a2-423e182e62b3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-08-11T07:06:10-07:00", + "end": "2020-08-11T07:06:10-07:00" + }, + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "395142003" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c", + "resource": { + "resourceType": "Encounter", + "id": "5aa23372-8c55-4ba4-80c6-37be3d22949c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2019-09-09T06:13:10-07:00", + "end": "2019-09-09T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:78f74aa9-e375-4759-aeb3-dd0b6559b3af", + "resource": { + "resourceType": "Observation", + "id": "78f74aa9-e375-4759-aeb3-dd0b6559b3af", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 78.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8233134-6384-4f59-90f8-bcb62762305d", + "resource": { + "resourceType": "Observation", + "id": "d8233134-6384-4f59-90f8-bcb62762305d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:891ee358-94ad-4f1c-8fb5-0277b633b591", + "resource": { + "resourceType": "Observation", + "id": "891ee358-94ad-4f1c-8fb5-0277b633b591", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 9.6000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59dc9644-adbc-44e4-9f03-deef443a1b98", + "resource": { + "resourceType": "Observation", + "id": "59dc9644-adbc-44e4-9f03-deef443a1b98", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 12.951, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ceeba545-7627-4585-b24b-3cd96903c26b", + "resource": { + "resourceType": "Observation", + "id": "ceeba545-7627-4585-b24b-3cd96903c26b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 46.970, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2e1cd59f-a823-404e-beb6-c80fcda809a3", + "resource": { + "resourceType": "Observation", + "id": "2e1cd59f-a823-404e-beb6-c80fcda809a3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:be2ffbce-f21b-4abc-983e-7b91f81d6390", + "resource": { + "resourceType": "Observation", + "id": "be2ffbce-f21b-4abc-983e-7b91f81d6390", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 72, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb29d6dd-badb-43ad-abed-4d18b39a4313", + "resource": { + "resourceType": "Observation", + "id": "fb29d6dd-badb-43ad-abed-4d18b39a4313", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d27a850-b426-466d-b8f8-5392dee55185", + "resource": { + "resourceType": "Observation", + "id": "3d27a850-b426-466d-b8f8-5392dee55185", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0163a714-c267-4559-ba24-4374d2f70bf3", + "resource": { + "resourceType": "Immunization", + "id": "0163a714-c267-4559-ba24-4374d2f70bf3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d2d20c61-a742-4d8f-b26e-da5e5b862cc5", + "resource": { + "resourceType": "Immunization", + "id": "d2d20c61-a742-4d8f-b26e-da5e5b862cc5", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2b9227b0-2535-4c7b-977c-6f9899ef53c2", + "resource": { + "resourceType": "Immunization", + "id": "2b9227b0-2535-4c7b-977c-6f9899ef53c2", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c57c49d3-5f3a-47e3-808f-fb6f315dea40", + "resource": { + "resourceType": "Immunization", + "id": "c57c49d3-5f3a-47e3-808f-fb6f315dea40", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:198993c4-73fd-4314-909d-3a0d9770434b", + "resource": { + "resourceType": "Immunization", + "id": "198993c4-73fd-4314-909d-3a0d9770434b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77", + "resource": { + "resourceType": "Claim", + "id": "77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-09-09T06:13:10-07:00", + "end": "2019-09-09T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:0163a714-c267-4559-ba24-4374d2f70bf3" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d2d20c61-a742-4d8f-b26e-da5e5b862cc5" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:2b9227b0-2535-4c7b-977c-6f9899ef53c2" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c57c49d3-5f3a-47e3-808f-fb6f315dea40" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:198993c4-73fd-4314-909d-3a0d9770434b" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3b0dd334-0e8a-4f21-8d88-c5c74168dd9b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3b0dd334-0e8a-4f21-8d88-c5c74168dd9b", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-09-09T06:28:10-07:00", + "end": "2020-09-09T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897", + "resource": { + "resourceType": "Encounter", + "id": "4a0a3706-4dec-4bd1-bcd2-04b43b9b5897", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:43:10-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e52b282f-d0db-4563-9fa6-b4c4fd9b2634", + "resource": { + "resourceType": "Observation", + "id": "e52b282f-d0db-4563-9fa6-b4c4fd9b2634", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 81.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:345104fe-7d72-401e-b86e-6e2c46332075", + "resource": { + "resourceType": "Observation", + "id": "345104fe-7d72-401e-b86e-6e2c46332075", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:144df2bb-df62-41db-95d4-bc817c50c768", + "resource": { + "resourceType": "Observation", + "id": "144df2bb-df62-41db-95d4-bc817c50c768", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 10.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:295341fa-d93c-43e6-bc77-020682f7da79", + "resource": { + "resourceType": "Observation", + "id": "295341fa-d93c-43e6-bc77-020682f7da79", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 9.5225, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e7c51d7b-3207-40a8-b286-d6c165c708fb", + "resource": { + "resourceType": "Observation", + "id": "e7c51d7b-3207-40a8-b286-d6c165c708fb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 47.610, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a73cf61b-f65d-4a99-9ba2-b30eb1ca230b", + "resource": { + "resourceType": "Observation", + "id": "a73cf61b-f65d-4a99-9ba2-b30eb1ca230b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2286c5f5-9bf5-4355-882b-e0639555fe49", + "resource": { + "resourceType": "Observation", + "id": "2286c5f5-9bf5-4355-882b-e0639555fe49", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9af05c49-221f-4c55-9551-cd580f69253d", + "resource": { + "resourceType": "Observation", + "id": "9af05c49-221f-4c55-9551-cd580f69253d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b19b739b-9393-4360-b700-7c6dc0c012c1", + "resource": { + "resourceType": "Observation", + "id": "b19b739b-9393-4360-b700-7c6dc0c012c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7adda0ad-c08a-4436-b059-c0d91b7fe86d", + "resource": { + "resourceType": "Procedure", + "id": "7adda0ad-c08a-4436-b059-c0d91b7fe86d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "performedPeriod": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:28:10-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:8fc177fe-6983-491a-abb3-c594e8095837", + "resource": { + "resourceType": "Immunization", + "id": "8fc177fe-6983-491a-abb3-c594e8095837", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "encounter": { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + }, + "date": "2019-12-09T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c6be629b-8271-42ec-a56d-a4e5318bca81", + "resource": { + "resourceType": "Claim", + "id": "c6be629b-8271-42ec-a56d-a4e5318bca81", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:43:10-08:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8fc177fe-6983-491a-abb3-c594e8095837" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7adda0ad-c08a-4436-b059-c0d91b7fe86d" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 671.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:61596942-9e73-4464-9cd7-893f73c722b8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "61596942-9e73-4464-9cd7-893f73c722b8", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c6be629b-8271-42ec-a56d-a4e5318bca81" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-12-09T05:43:10-08:00", + "end": "2020-12-09T05:43:10-08:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 134.346, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 537.384, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 671.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 671.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 649.8000000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3", + "resource": { + "resourceType": "Encounter", + "id": "b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308646001", + "display": "Death Certification" + } + ], + "text": "Death Certification" + } + ], + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" + } + } + ], + "period": { + "start": "2020-04-13T06:13:10-07:00", + "end": "2020-04-13T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4a076392-7755-4007-bd4e-516b124cceb5", + "resource": { + "resourceType": "Observation", + "id": "4a076392-7755-4007-bd4e-516b124cceb5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "exam", + "display": "exam" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "69453-9", + "display": "Cause of Death [US Standard Certificate of Death]" + } + ], + "text": "Cause of Death [US Standard Certificate of Death]" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" + }, + "effectiveDateTime": "2020-04-13T06:13:10-07:00", + "issued": "2020-04-13T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "95281009", + "display": "Sudden Cardiac Death" + } + ], + "text": "Sudden Cardiac Death" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cef6a500-0ea0-41fe-8606-bf3221180ba7", + "resource": { + "resourceType": "DiagnosticReport", + "id": "cef6a500-0ea0-41fe-8606-bf3221180ba7", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "69409-1", + "display": "U.S. standard certificate of death - 2003 revision" + } + ], + "text": "U.S. standard certificate of death - 2003 revision" + }, + "subject": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "context": { + "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" + }, + "effectiveDateTime": "2020-04-13T06:13:10-07:00", + "issued": "2020-04-13T06:13:10.107-07:00", + "result": [ + { + "reference": "urn:uuid:4a076392-7755-4007-bd4e-516b124cceb5", + "display": "Cause of Death [US Standard Certificate of Death]" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:100daf7c-469d-40e7-b263-184f2f44d223", + "resource": { + "resourceType": "Claim", + "id": "100daf7c-469d-40e7-b263-184f2f44d223", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "start": "2020-04-13T06:13:10-07:00", + "end": "2020-04-13T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0c772652-8ab7-4e11-a049-a86d04c2bbb5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0c772652-8ab7-4e11-a049-a86d04c2bbb5", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "100daf7c-469d-40e7-b263-184f2f44d223" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2020-04-13T06:28:10-07:00", + "end": "2021-04-13T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" + } + }, + "organization": { + "identifier": { + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json new file mode 100644 index 000000000000..1bf7f49aa84b --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json @@ -0,0 +1,15372 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6", + "resource": { + "resourceType": "Patient", + "id": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -3160153991630561462 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2054-5", + "display": "Black or African American" + } + }, + { + "url": "text", + "valueString": "Black or African American" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Asia282 Crona259" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "M" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Wellesley", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.0 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 1.0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-12-5406" + } + ], + "name": [ + { + "use": "official", + "family": "Pfannerstill264", + "given": [ + "Basil991" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-373-9418", + "use": "home" + } + ], + "gender": "male", + "birthDate": "2018-07-02", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.40406475290831 + }, + { + "url": "longitude", + "valueDecimal": -71.12753835800636 + } + ] + } + ], + "line": [ + "720 Hickle Light Apt 14" + ], + "city": "Cambridge", + "state": "Massachusetts", + "postalCode": "02472", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212", + "resource": { + "resourceType": "Organization", + "id": "83284578-12e5-3582-bfe1-7807e3f7a212", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP297713", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.423844 + }, + { + "url": "longitude", + "valueDecimal": -71.10923100000001 + } + ] + } + ], + "line": [ + "0 GOVERNORS AVE" + ], + "city": "MEDFORD", + "state": "MA", + "postalCode": "02155-3084", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499", + "resource": { + "resourceType": "Practitioner", + "id": "d726e757-2975-3d6a-bdf9-517546432499", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "80690" + } + ], + "active": true, + "name": [ + { + "family": "Abernathy524", + "given": [ + "Ulrike696" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "0 GOVERNORS AVE" + ], + "city": "MEDFORD", + "state": "MA", + "postalCode": "02155-3084", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a", + "resource": { + "resourceType": "Encounter", + "id": "41d9c565-feb8-49b7-8dc8-c4900f98793a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3f83f9ad-288b-43a2-a4f2-7c6fb421f525", + "resource": { + "resourceType": "Observation", + "id": "3f83f9ad-288b-43a2-a4f2-7c6fb421f525", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 50.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51654efa-2ee3-4acb-9788-12a8da445dc3", + "resource": { + "resourceType": "Observation", + "id": "51654efa-2ee3-4acb-9788-12a8da445dc3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6015ade7-061d-4fcd-bbf9-0e3c7b5342f1", + "resource": { + "resourceType": "Observation", + "id": "6015ade7-061d-4fcd-bbf9-0e3c7b5342f1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 3.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9357cf13-166e-4e60-a73f-6be78861709d", + "resource": { + "resourceType": "Observation", + "id": "9357cf13-166e-4e60-a73f-6be78861709d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 41.793, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8c89cd9d-b297-474e-bb30-9b79917684c8", + "resource": { + "resourceType": "Observation", + "id": "8c89cd9d-b297-474e-bb30-9b79917684c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 33.670, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9603a9c5-a600-472d-aaa4-2c618566a50e", + "resource": { + "resourceType": "Observation", + "id": "9603a9c5-a600-472d-aaa4-2c618566a50e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 104, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:59171c7d-b931-44f0-b0aa-117a6520d463", + "resource": { + "resourceType": "Observation", + "id": "59171c7d-b931-44f0-b0aa-117a6520d463", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 84, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:483fa61e-36df-4205-ad60-dbecca018ff5", + "resource": { + "resourceType": "Observation", + "id": "483fa61e-36df-4205-ad60-dbecca018ff5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:305ca67d-1925-472b-b350-a8997785402d", + "resource": { + "resourceType": "Observation", + "id": "305ca67d-1925-472b-b350-a8997785402d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 8.0387, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b48ddff0-b112-4074-a71e-9fa48369ea8f", + "resource": { + "resourceType": "Observation", + "id": "b48ddff0-b112-4074-a71e-9fa48369ea8f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 4.2691, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1428d255-36bc-4c2c-bfd0-07b812870867", + "resource": { + "resourceType": "Observation", + "id": "1428d255-36bc-4c2c-bfd0-07b812870867", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 13.901, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:afff4d3d-b6ca-40bd-8eee-e72996840ead", + "resource": { + "resourceType": "Observation", + "id": "afff4d3d-b6ca-40bd-8eee-e72996840ead", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 40.693, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b4e5944-6741-434c-b419-4a428d31919f", + "resource": { + "resourceType": "Observation", + "id": "2b4e5944-6741-434c-b419-4a428d31919f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 80.501, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", + "resource": { + "resourceType": "Observation", + "id": "9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 30.643, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:64801d0a-09b0-41fc-90bc-5ec827640cca", + "resource": { + "resourceType": "Observation", + "id": "64801d0a-09b0-41fc-90bc-5ec827640cca", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 33.848, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:22f92c59-c8cc-4d52-9ada-53a218396179", + "resource": { + "resourceType": "Observation", + "id": "22f92c59-c8cc-4d52-9ada-53a218396179", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 42.600, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", + "resource": { + "resourceType": "Observation", + "id": "2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 420.43, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:350edd3f-f32a-4b5c-b05a-f1e26b944f8a", + "resource": { + "resourceType": "Observation", + "id": "350edd3f-f32a-4b5c-b05a-f1e26b944f8a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 233.63, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12d5bae9-94a3-458a-a398-b49bfa340628", + "resource": { + "resourceType": "Observation", + "id": "12d5bae9-94a3-458a-a398-b49bfa340628", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueQuantity": { + "value": 9.4122, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5e895c0-210b-43bb-b2cb-e7c539768cf8", + "resource": { + "resourceType": "Observation", + "id": "f5e895c0-210b-43bb-b2cb-e7c539768cf8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a28f96b8-1a93-473a-9454-dccb88955db7", + "resource": { + "resourceType": "Procedure", + "id": "a28f96b8-1a93-473a-9454-dccb88955db7", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "performedPeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c27f8d19-48e1-4739-89fc-141e1a9d7ce6", + "resource": { + "resourceType": "Immunization", + "id": "c27f8d19-48e1-4739-89fc-141e1a9d7ce6", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "date": "2018-07-02T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a0fac80f-483e-41f2-b361-ca919e400c71", + "resource": { + "resourceType": "DiagnosticReport", + "id": "a0fac80f-483e-41f2-b361-ca919e400c71", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + }, + "effectiveDateTime": "2018-07-02T06:13:10-07:00", + "issued": "2018-07-02T06:13:10.107-07:00", + "result": [ + { + "reference": "urn:uuid:305ca67d-1925-472b-b350-a8997785402d", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b48ddff0-b112-4074-a71e-9fa48369ea8f", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1428d255-36bc-4c2c-bfd0-07b812870867", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:afff4d3d-b6ca-40bd-8eee-e72996840ead", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:2b4e5944-6741-434c-b419-4a428d31919f", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:64801d0a-09b0-41fc-90bc-5ec827640cca", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:22f92c59-c8cc-4d52-9ada-53a218396179", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:350edd3f-f32a-4b5c-b05a-f1e26b944f8a", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:12d5bae9-94a3-458a-a398-b49bfa340628", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:1a73022b-ab53-496e-8ec4-fd11d353a97a", + "resource": { + "resourceType": "Claim", + "id": "1a73022b-ab53-496e-8ec4-fd11d353a97a", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2018-07-02T06:13:10-07:00", + "end": "2018-07-02T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c27f8d19-48e1-4739-89fc-141e1a9d7ce6" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a28f96b8-1a93-473a-9454-dccb88955db7" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 510.48, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:664de9a4-da89-4727-924a-f6ab9ff52686", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "664de9a4-da89-4727-924a-f6ab9ff52686", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1a73022b-ab53-496e-8ec4-fd11d353a97a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-07-02T06:43:10-07:00", + "end": "2019-07-02T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 102.096, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 408.384, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 510.48, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 510.48, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 520.8000000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066", + "resource": { + "resourceType": "Encounter", + "id": "2b2dc0de-65d8-4673-9e28-ee5c85a3d066", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2018-08-06T06:13:10-07:00", + "end": "2018-08-06T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c423e214-b4f6-4601-bf66-729a07205e48", + "resource": { + "resourceType": "Observation", + "id": "c423e214-b4f6-4601-bf66-729a07205e48", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 54.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:acd9d21d-c3f6-4704-a1e0-892946ddc950", + "resource": { + "resourceType": "Observation", + "id": "acd9d21d-c3f6-4704-a1e0-892946ddc950", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9017713d-30ab-4825-be7a-448339eeaf05", + "resource": { + "resourceType": "Observation", + "id": "9017713d-30ab-4825-be7a-448339eeaf05", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 4.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fe916404-d864-4d5b-a2be-9aca37e28687", + "resource": { + "resourceType": "Observation", + "id": "fe916404-d864-4d5b-a2be-9aca37e28687", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 25.763, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf8939f8-e35d-4977-8464-567af8adc3c2", + "resource": { + "resourceType": "Observation", + "id": "cf8939f8-e35d-4977-8464-567af8adc3c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 37.430, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:146779d7-66ba-4005-b989-5c2a5365d422", + "resource": { + "resourceType": "Observation", + "id": "146779d7-66ba-4005-b989-5c2a5365d422", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 70, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 134, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3a5215d7-fcef-45e3-9d25-91d5a8d13df5", + "resource": { + "resourceType": "Observation", + "id": "3a5215d7-fcef-45e3-9d25-91d5a8d13df5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 84, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1dad280f-c7fe-40d0-9bbe-13fa34b9743c", + "resource": { + "resourceType": "Observation", + "id": "1dad280f-c7fe-40d0-9bbe-13fa34b9743c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1604fe32-413e-4571-b1a1-555ad84b9cdf", + "resource": { + "resourceType": "Observation", + "id": "1604fe32-413e-4571-b1a1-555ad84b9cdf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "effectiveDateTime": "2018-08-06T06:13:10-07:00", + "issued": "2018-08-06T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:362ab788-f630-4514-9efa-433dee3edf2e", + "resource": { + "resourceType": "Immunization", + "id": "362ab788-f630-4514-9efa-433dee3edf2e", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + }, + "date": "2018-08-06T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d55e5479-6f83-4d8d-969a-63e18cefeede", + "resource": { + "resourceType": "Claim", + "id": "d55e5479-6f83-4d8d-969a-63e18cefeede", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2018-08-06T06:13:10-07:00", + "end": "2018-08-06T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:362ab788-f630-4514-9efa-433dee3edf2e" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:30538e24-8919-458b-9241-b49073124182", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "30538e24-8919-458b-9241-b49073124182", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d55e5479-6f83-4d8d-969a-63e18cefeede" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-08-06T06:28:10-07:00", + "end": "2019-08-06T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1", + "resource": { + "resourceType": "Encounter", + "id": "e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2018-10-08T06:13:10-07:00", + "end": "2018-10-08T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:be531e9a-d7db-4ec9-81d8-e827032c3752", + "resource": { + "resourceType": "Observation", + "id": "be531e9a-d7db-4ec9-81d8-e827032c3752", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 59.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4569557d-7c4e-4ece-beef-a3909ed9587c", + "resource": { + "resourceType": "Observation", + "id": "4569557d-7c4e-4ece-beef-a3909ed9587c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1261b413-92b1-45de-b4b9-d450c566fe7a", + "resource": { + "resourceType": "Observation", + "id": "1261b413-92b1-45de-b4b9-d450c566fe7a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 5.7000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:980ef229-efdf-417e-9bd3-f7700a6bb531", + "resource": { + "resourceType": "Observation", + "id": "980ef229-efdf-417e-9bd3-f7700a6bb531", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 25.701, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f27810b-aa48-45de-8346-52a7f456aa7e", + "resource": { + "resourceType": "Observation", + "id": "1f27810b-aa48-45de-8346-52a7f456aa7e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 40.230, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:471e0d21-a483-40dc-97d7-3df908d031aa", + "resource": { + "resourceType": "Observation", + "id": "471e0d21-a483-40dc-97d7-3df908d031aa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fbafe674-f215-4f9d-a639-6208e0abf44f", + "resource": { + "resourceType": "Observation", + "id": "fbafe674-f215-4f9d-a639-6208e0abf44f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 83, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36ff5081-58eb-42e6-a119-01712beb9f68", + "resource": { + "resourceType": "Observation", + "id": "36ff5081-58eb-42e6-a119-01712beb9f68", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d75171f-8af8-447f-9bc5-5b2a12e7d7dc", + "resource": { + "resourceType": "Observation", + "id": "9d75171f-8af8-447f-9bc5-5b2a12e7d7dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "effectiveDateTime": "2018-10-08T06:13:10-07:00", + "issued": "2018-10-08T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5b63d12-9e5c-47e5-a887-0a01c14ca57a", + "resource": { + "resourceType": "Procedure", + "id": "a5b63d12-9e5c-47e5-a887-0a01c14ca57a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "performedPeriod": { + "start": "2018-10-08T06:13:10-07:00", + "end": "2018-10-08T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:628d5705-ccb9-43e8-970a-4566b7a0e319", + "resource": { + "resourceType": "Immunization", + "id": "628d5705-ccb9-43e8-970a-4566b7a0e319", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7e1a73c9-11eb-4b4f-979f-9b1382a47db3", + "resource": { + "resourceType": "Immunization", + "id": "7e1a73c9-11eb-4b4f-979f-9b1382a47db3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d3b80ab6-a01d-418e-aba5-11199e756a66", + "resource": { + "resourceType": "Immunization", + "id": "d3b80ab6-a01d-418e-aba5-11199e756a66", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:304b49f5-521e-4f6a-b9cc-9c74ce0e3292", + "resource": { + "resourceType": "Immunization", + "id": "304b49f5-521e-4f6a-b9cc-9c74ce0e3292", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:173ca7d0-b5f4-43d1-8efa-3826118fed63", + "resource": { + "resourceType": "Immunization", + "id": "173ca7d0-b5f4-43d1-8efa-3826118fed63", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + }, + "date": "2018-10-08T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2ee27832-be87-4039-8d48-413bdf2c02db", + "resource": { + "resourceType": "Claim", + "id": "2ee27832-be87-4039-8d48-413bdf2c02db", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2018-10-08T06:13:10-07:00", + "end": "2018-10-08T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:628d5705-ccb9-43e8-970a-4566b7a0e319" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7e1a73c9-11eb-4b4f-979f-9b1382a47db3" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d3b80ab6-a01d-418e-aba5-11199e756a66" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:304b49f5-521e-4f6a-b9cc-9c74ce0e3292" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:173ca7d0-b5f4-43d1-8efa-3826118fed63" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a5b63d12-9e5c-47e5-a887-0a01c14ca57a" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 575.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:624d52c3-267c-479c-9970-bf95e97c525d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "624d52c3-267c-479c-9970-bf95e97c525d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2ee27832-be87-4039-8d48-413bdf2c02db" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-10-08T06:43:10-07:00", + "end": "2019-10-08T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 115.14000000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 460.56000000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 575.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 575.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1022.6400000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f", + "resource": { + "resourceType": "Encounter", + "id": "eefc8fce-bc6b-41a3-b6d1-f731bb560f0f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:43:10-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:05a40880-1366-40af-95c1-5966694a5bdc", + "resource": { + "resourceType": "Observation", + "id": "05a40880-1366-40af-95c1-5966694a5bdc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 63.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:62c1a0fc-fac2-4af0-acc4-35f9fda22498", + "resource": { + "resourceType": "Observation", + "id": "62c1a0fc-fac2-4af0-acc4-35f9fda22498", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a137e5d6-3956-4436-877d-143c29966fb1", + "resource": { + "resourceType": "Observation", + "id": "a137e5d6-3956-4436-877d-143c29966fb1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 6.8000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:461ae86e-efd4-46cf-8f95-9b97b129edde", + "resource": { + "resourceType": "Observation", + "id": "461ae86e-efd4-46cf-8f95-9b97b129edde", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 32.712, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0f029843-14f7-4678-8766-66bd96bdaf5f", + "resource": { + "resourceType": "Observation", + "id": "0f029843-14f7-4678-8766-66bd96bdaf5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 41.990, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:01ac0323-eab5-406a-ad6c-8dfaa12d5b72", + "resource": { + "resourceType": "Observation", + "id": "01ac0323-eab5-406a-ad6c-8dfaa12d5b72", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 102, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e1a89a17-3a00-4783-8a5f-053a052fadaa", + "resource": { + "resourceType": "Observation", + "id": "e1a89a17-3a00-4783-8a5f-053a052fadaa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:38264da7-a579-46d7-aee1-6a44c423e75d", + "resource": { + "resourceType": "Observation", + "id": "38264da7-a579-46d7-aee1-6a44c423e75d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:56023140-1737-445a-b834-49a9b0a932a6", + "resource": { + "resourceType": "Observation", + "id": "56023140-1737-445a-b834-49a9b0a932a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "effectiveDateTime": "2018-12-10T05:13:10-08:00", + "issued": "2018-12-10T05:13:10.107-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2b3f5fd8-7a23-4a20-a571-7287f33e8a12", + "resource": { + "resourceType": "Procedure", + "id": "2b3f5fd8-7a23-4a20-a571-7287f33e8a12", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "performedPeriod": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:28:10-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec", + "resource": { + "resourceType": "Immunization", + "id": "9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f85344ff-d6ed-4c2f-b671-3755d4b0c614", + "resource": { + "resourceType": "Immunization", + "id": "f85344ff-d6ed-4c2f-b671-3755d4b0c614", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1dba5b1b-ef05-4742-b4d8-3c780bcd2439", + "resource": { + "resourceType": "Immunization", + "id": "1dba5b1b-ef05-4742-b4d8-3c780bcd2439", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b93bf8c9-925d-4bfa-838c-25240ef7ff61", + "resource": { + "resourceType": "Immunization", + "id": "b93bf8c9-925d-4bfa-838c-25240ef7ff61", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:33c13591-1d77-42a1-b519-5ecbea921a2e", + "resource": { + "resourceType": "Immunization", + "id": "33c13591-1d77-42a1-b519-5ecbea921a2e", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + }, + "date": "2018-12-10T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:fca0a350-61ec-49d4-8bbd-cd951e53570b", + "resource": { + "resourceType": "Claim", + "id": "fca0a350-61ec-49d4-8bbd-cd951e53570b", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2018-12-10T05:13:10-08:00", + "end": "2018-12-10T05:43:10-08:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f85344ff-d6ed-4c2f-b671-3755d4b0c614" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1dba5b1b-ef05-4742-b4d8-3c780bcd2439" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:b93bf8c9-925d-4bfa-838c-25240ef7ff61" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:33c13591-1d77-42a1-b519-5ecbea921a2e" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:2b3f5fd8-7a23-4a20-a571-7287f33e8a12" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 461.41, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:75de1d08-9852-400a-a42e-108e938c26bb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "75de1d08-9852-400a-a42e-108e938c26bb", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fca0a350-61ec-49d4-8bbd-cd951e53570b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-12-10T05:43:10-08:00", + "end": "2019-12-10T05:43:10-08:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 92.28200000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 369.12800000000004, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 461.41, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 461.41, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 931.2080000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578", + "resource": { + "resourceType": "Encounter", + "id": "cd9fa994-4899-4e33-bcbd-7e526b471578", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2019-03-11T06:13:10-07:00", + "end": "2019-03-11T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7af69df3-a78a-41d6-9fd8-2fc36529bc08", + "resource": { + "resourceType": "Observation", + "id": "7af69df3-a78a-41d6-9fd8-2fc36529bc08", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 68.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0fa679e1-97d1-46a4-8afa-cbf6a8f52886", + "resource": { + "resourceType": "Observation", + "id": "0fa679e1-97d1-46a4-8afa-cbf6a8f52886", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:18588a86-37e4-4701-b9a2-722f41b13c6b", + "resource": { + "resourceType": "Observation", + "id": "18588a86-37e4-4701-b9a2-722f41b13c6b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 8.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1fbf8520-879e-4ffc-b33e-bb88e5c1a833", + "resource": { + "resourceType": "Observation", + "id": "1fbf8520-879e-4ffc-b33e-bb88e5c1a833", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 58.512, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7949dbd0-ddf4-4105-aa2e-6d0e7e598a53", + "resource": { + "resourceType": "Observation", + "id": "7949dbd0-ddf4-4105-aa2e-6d0e7e598a53", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 43.730, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:61dd1c35-95dd-4b4a-9ff6-1b9e3a4a8271", + "resource": { + "resourceType": "Observation", + "id": "61dd1c35-95dd-4b4a-9ff6-1b9e3a4a8271", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cc0debc9-3366-49a6-9a9d-5b706adc56c3", + "resource": { + "resourceType": "Observation", + "id": "cc0debc9-3366-49a6-9a9d-5b706adc56c3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 65, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:685abc52-deb4-4ae7-a527-5bf07f875b3d", + "resource": { + "resourceType": "Observation", + "id": "685abc52-deb4-4ae7-a527-5bf07f875b3d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:27d7edef-1085-4075-a8a2-6d3b0e2db061", + "resource": { + "resourceType": "Observation", + "id": "27d7edef-1085-4075-a8a2-6d3b0e2db061", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "effectiveDateTime": "2019-03-11T06:13:10-07:00", + "issued": "2019-03-11T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6280d60b-fcf7-4a00-abc3-54ca2dba7375", + "resource": { + "resourceType": "Immunization", + "id": "6280d60b-fcf7-4a00-abc3-54ca2dba7375", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7329d302-1187-4106-a820-36e7e7fa0be8", + "resource": { + "resourceType": "Immunization", + "id": "7329d302-1187-4106-a820-36e7e7fa0be8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:112db251-367d-4e24-b5f0-74d347c20fc7", + "resource": { + "resourceType": "Immunization", + "id": "112db251-367d-4e24-b5f0-74d347c20fc7", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:5f503634-27b4-4d39-af2d-9616305fb112", + "resource": { + "resourceType": "Immunization", + "id": "5f503634-27b4-4d39-af2d-9616305fb112", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:09afdb46-413f-439e-a4d3-cc17e70540de", + "resource": { + "resourceType": "Immunization", + "id": "09afdb46-413f-439e-a4d3-cc17e70540de", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + }, + "date": "2019-03-11T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c7f1547d-b417-4360-9b4a-17e407b1799f", + "resource": { + "resourceType": "Claim", + "id": "c7f1547d-b417-4360-9b4a-17e407b1799f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-03-11T06:13:10-07:00", + "end": "2019-03-11T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:6280d60b-fcf7-4a00-abc3-54ca2dba7375" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7329d302-1187-4106-a820-36e7e7fa0be8" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:112db251-367d-4e24-b5f0-74d347c20fc7" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:5f503634-27b4-4d39-af2d-9616305fb112" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:09afdb46-413f-439e-a4d3-cc17e70540de" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a66e53c0-1be1-408d-adf9-7dbf08ce8850", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a66e53c0-1be1-408d-adf9-7dbf08ce8850", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c7f1547d-b417-4360-9b4a-17e407b1799f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-03-11T06:28:10-07:00", + "end": "2020-03-11T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3", + "resource": { + "resourceType": "Encounter", + "id": "dc6f80e7-beff-4b8a-8f62-b3e13ee315b3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2019-06-10T06:13:10-07:00", + "end": "2019-06-10T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:36c0832b-c32a-440a-a794-03f58d056ef4", + "resource": { + "resourceType": "Observation", + "id": "36c0832b-c32a-440a-a794-03f58d056ef4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 72, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2086c21a-a5af-4559-9420-92e13e76c294", + "resource": { + "resourceType": "Observation", + "id": "2086c21a-a5af-4559-9420-92e13e76c294", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:368d1e69-17b8-4c7a-b464-fe9e14711856", + "resource": { + "resourceType": "Observation", + "id": "368d1e69-17b8-4c7a-b464-fe9e14711856", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 9.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9b277d04-6184-4d37-9fea-10c2699fb028", + "resource": { + "resourceType": "Observation", + "id": "9b277d04-6184-4d37-9fea-10c2699fb028", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 59.098, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e2dc12e-15b0-4c52-ad97-9116e8f3dd4f", + "resource": { + "resourceType": "Observation", + "id": "1e2dc12e-15b0-4c52-ad97-9116e8f3dd4f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 44.880, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:73edd7e7-c6c3-4dc9-82b8-d162e2099271", + "resource": { + "resourceType": "Observation", + "id": "73edd7e7-c6c3-4dc9-82b8-d162e2099271", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:78603cd5-5300-401e-b2fe-157efd7b41b3", + "resource": { + "resourceType": "Observation", + "id": "78603cd5-5300-401e-b2fe-157efd7b41b3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 69, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:451028a6-f90c-41de-a525-3047815de7fe", + "resource": { + "resourceType": "Observation", + "id": "451028a6-f90c-41de-a525-3047815de7fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf54424e-c9b0-4c0d-ad50-3014dc5d3667", + "resource": { + "resourceType": "Observation", + "id": "cf54424e-c9b0-4c0d-ad50-3014dc5d3667", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "effectiveDateTime": "2019-06-10T06:13:10-07:00", + "issued": "2019-06-10T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fe4b7981-bd87-4d0b-82d3-c168ba8c448a", + "resource": { + "resourceType": "Procedure", + "id": "fe4b7981-bd87-4d0b-82d3-c168ba8c448a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + }, + "performedPeriod": { + "start": "2019-06-10T06:13:10-07:00", + "end": "2019-06-10T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:bf665789-c5c6-452d-9363-03d077ab69b0", + "resource": { + "resourceType": "Claim", + "id": "bf665789-c5c6-452d-9363-03d077ab69b0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-06-10T06:13:10-07:00", + "end": "2019-06-10T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:fe4b7981-bd87-4d0b-82d3-c168ba8c448a" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 690.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:37373884-0dde-4dd8-81c9-4967c8315b53", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "37373884-0dde-4dd8-81c9-4967c8315b53", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bf665789-c5c6-452d-9363-03d077ab69b0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-06-10T06:43:10-07:00", + "end": "2020-06-10T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 138.1, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 552.4, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 690.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 690.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 552.4, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", + "resource": { + "resourceType": "Practitioner", + "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10" + } + ], + "active": true, + "name": [ + { + "family": "Jenkins714", + "given": [ + "Lara964" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71", + "resource": { + "resourceType": "Encounter", + "id": "601f35fa-fb17-4771-a033-d1b624db5a71", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5", + "resource": { + "resourceType": "Condition", + "id": "8a2d671f-09cc-4e92-89aa-27e19adb0ea5", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "241929008", + "display": "Acute allergic reaction" + } + ], + "text": "Acute allergic reaction" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + }, + "onsetDateTime": "2019-08-26T06:13:10-07:00", + "abatementDateTime": "2019-08-26T11:13:10-07:00", + "assertedDate": "2019-08-26T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:cd31a507-9600-4978-b8e2-0fecb70d07a5", + "resource": { + "resourceType": "Procedure", + "id": "cd31a507-9600-4978-b8e2-0fecb70d07a5", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "313191000", + "display": "Injection of adrenaline" + } + ], + "text": "Injection of adrenaline" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + }, + "performedPeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5d47da3e-ada9-48eb-8199-7d1540239fdb", + "resource": { + "resourceType": "MedicationRequest", + "id": "5d47da3e-ada9-48eb-8199-7d1540239fdb", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "312617", + "display": "predniSONE 5 MG Oral Tablet" + } + ], + "text": "predniSONE 5 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + }, + "authoredOn": "2019-08-26T11:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:3e823594-104b-48e5-a2b5-92f5da19f176", + "resource": { + "resourceType": "Claim", + "id": "3e823594-104b-48e5-a2b5-92f5da19f176", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:5d47da3e-ada9-48eb-8199-7d1540239fdb" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + } + ] + } + ], + "total": { + "value": 7.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f5fe4b03-66c7-4f84-84e8-4de5823e9d7d", + "resource": { + "resourceType": "Claim", + "id": "f5fe4b03-66c7-4f84-84e8-4de5823e9d7d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:cd31a507-9600-4978-b8e2-0fecb70d07a5" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "313191000" + } + ] + }, + "net": { + "value": 2333.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5f31fcea-92f1-45dc-9d4a-605208e041a7", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5f31fcea-92f1-45dc-9d4a-605208e041a7", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f5fe4b03-66c7-4f84-84e8-4de5823e9d7d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-08-26T11:28:10-07:00", + "end": "2020-08-26T11:28:10-07:00" + }, + "created": "2019-08-26T11:28:10-07:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:f5fe4b03-66c7-4f84-84e8-4de5823e9d7d" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "313191000" + } + ] + }, + "servicedPeriod": { + "start": "2019-08-26T06:13:10-07:00", + "end": "2019-08-26T11:28:10-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "net": { + "value": 2333.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 466.65200000000004, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1866.6080000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2333.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2333.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1866.6080000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "resource": { + "resourceType": "Organization", + "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAMBRIDGE HEALTH ALLIANCE", + "telecom": [ + { + "system": "phone", + "value": "6176652300" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", + "resource": { + "resourceType": "Practitioner", + "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "40" + } + ], + "active": true, + "name": [ + { + "family": "Gibson10", + "given": [ + "Loretta235" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab", + "resource": { + "resourceType": "Encounter", + "id": "cb9f6247-9e30-4bff-90c1-34d7c66fa6ab", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2019-09-02T06:13:10-07:00", + "end": "2019-09-02T06:28:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8cef7d56-00a9-4758-b00b-2e3625e04acc", + "resource": { + "resourceType": "CarePlan", + "id": "8cef7d56-00a9-4758-b00b-2e3625e04acc", + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "384758001", + "display": "Self-care interventions (procedure)" + } + ], + "text": "Self-care interventions (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab" + }, + "period": { + "start": "2019-09-02T06:13:10-07:00" + }, + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "409002", + "display": "Food allergy diet" + } + ], + "text": "Food allergy diet" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58332002", + "display": "Allergy education" + } + ], + "text": "Allergy education" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "58332002", + "display": "Allergy education" + } + ], + "text": "Allergy education" + }, + "status": "in-progress" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:f554de70-4dcd-4bd2-a95d-1490f2079800", + "resource": { + "resourceType": "Claim", + "id": "f554de70-4dcd-4bd2-a95d-1490f2079800", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-09-02T06:13:10-07:00", + "end": "2019-09-02T06:28:10-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9e48ce54-a7e9-448d-9d76-4e522fabfa85", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "9e48ce54-a7e9-448d-9d76-4e522fabfa85", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f554de70-4dcd-4bd2-a95d-1490f2079800" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-09-02T06:28:10-07:00", + "end": "2020-09-02T06:28:10-07:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71", + "resource": { + "resourceType": "Encounter", + "id": "ed001667-14dd-48f1-a5ce-0bfb8b373d71", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2019-09-09T06:13:10-07:00", + "end": "2019-09-09T06:43:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2726d223-c231-4e88-a777-0d040011b25e", + "resource": { + "resourceType": "Observation", + "id": "2726d223-c231-4e88-a777-0d040011b25e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 75.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25144b62-082e-4e72-ad67-fde8c6689436", + "resource": { + "resourceType": "Observation", + "id": "25144b62-082e-4e72-ad67-fde8c6689436", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed796f59-3291-4107-9dca-dc11e97044da", + "resource": { + "resourceType": "Observation", + "id": "ed796f59-3291-4107-9dca-dc11e97044da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 9.9000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32934caf-6089-444c-b980-7a0b89afd141", + "resource": { + "resourceType": "Observation", + "id": "32934caf-6089-444c-b980-7a0b89afd141", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 58.121, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5dd8046f-66cc-4111-a89c-72f4665cf3f6", + "resource": { + "resourceType": "Observation", + "id": "5dd8046f-66cc-4111-a89c-72f4665cf3f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 45.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7753fc67-7ae2-4681-b6bc-abc53286acc4", + "resource": { + "resourceType": "Observation", + "id": "7753fc67-7ae2-4681-b6bc-abc53286acc4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d81f759f-a40b-46bc-9e7b-9b77c368cbdc", + "resource": { + "resourceType": "Observation", + "id": "d81f759f-a40b-46bc-9e7b-9b77c368cbdc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 86, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f950fb54-7057-4898-aa24-03e549d51b0c", + "resource": { + "resourceType": "Observation", + "id": "f950fb54-7057-4898-aa24-03e549d51b0c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82e1d551-0027-4974-ae34-ef54ec72f4c7", + "resource": { + "resourceType": "Observation", + "id": "82e1d551-0027-4974-ae34-ef54ec72f4c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "effectiveDateTime": "2019-09-09T06:13:10-07:00", + "issued": "2019-09-09T06:13:10.107-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:739c6bf8-1a80-40e1-a07e-74da5797c929", + "resource": { + "resourceType": "Procedure", + "id": "739c6bf8-1a80-40e1-a07e-74da5797c929", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "performedPeriod": { + "start": "2019-09-09T06:13:10-07:00", + "end": "2019-09-09T06:28:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:bf9f65db-9e55-490b-a69a-4ca965e7e393", + "resource": { + "resourceType": "Immunization", + "id": "bf9f65db-9e55-490b-a69a-4ca965e7e393", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:169a63ab-2a78-4c8f-a4ca-a665103f02b1", + "resource": { + "resourceType": "Immunization", + "id": "169a63ab-2a78-4c8f-a4ca-a665103f02b1", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:36c925c2-10aa-47fe-8c70-78428322704a", + "resource": { + "resourceType": "Immunization", + "id": "36c925c2-10aa-47fe-8c70-78428322704a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:46dd2af4-dcb4-46a4-9f50-3dfb0e632b88", + "resource": { + "resourceType": "Immunization", + "id": "46dd2af4-dcb4-46a4-9f50-3dfb0e632b88", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:72bf6a64-946e-45bb-9919-94542296ba16", + "resource": { + "resourceType": "Immunization", + "id": "72bf6a64-946e-45bb-9919-94542296ba16", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + }, + "date": "2019-09-09T06:13:10-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4fddfb60-257f-4f89-ac0a-dc1b6ca24076", + "resource": { + "resourceType": "Claim", + "id": "4fddfb60-257f-4f89-ac0a-dc1b6ca24076", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-09-09T06:13:10-07:00", + "end": "2019-09-09T06:43:10-07:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:bf9f65db-9e55-490b-a69a-4ca965e7e393" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:169a63ab-2a78-4c8f-a4ca-a665103f02b1" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:36c925c2-10aa-47fe-8c70-78428322704a" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:46dd2af4-dcb4-46a4-9f50-3dfb0e632b88" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:72bf6a64-946e-45bb-9919-94542296ba16" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:739c6bf8-1a80-40e1-a07e-74da5797c929" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 673.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:cbfd86b3-3df7-42d3-9ddb-32e6df99de6d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "cbfd86b3-3df7-42d3-9ddb-32e6df99de6d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4fddfb60-257f-4f89-ac0a-dc1b6ca24076" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-09-09T06:43:10-07:00", + "end": "2020-09-09T06:43:10-07:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 134.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 539.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 673.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 673.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1101.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7", + "resource": { + "resourceType": "Encounter", + "id": "3b95a3d5-eb06-4983-990d-2bcc641a43a7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185347001", + "display": "Encounter for problem" + } + ], + "text": "Encounter for problem" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2019-09-14T06:13:10-07:00", + "end": "2019-09-14T06:49:10-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d4d80701-e1e6-4656-bda3-a1770afb5122", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "d4d80701-e1e6-4656-bda3-a1770afb5122", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "419474003", + "display": "Allergy to mould" + } + ], + "text": "Allergy to mould" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:f8b3b9f1-b2df-4434-9369-227d1a92e18c", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "f8b3b9f1-b2df-4434-9369-227d1a92e18c", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "232350006", + "display": "House dust mite allergy" + } + ], + "text": "House dust mite allergy" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:12b4987e-75d2-457f-88d1-6fb5e04097a8", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "12b4987e-75d2-457f-88d1-6fb5e04097a8", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "232347008", + "display": "Dander (animal) allergy" + } + ], + "text": "Dander (animal) allergy" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:f9c3dd3f-0b25-4c6b-88c1-f5e73213c6dc", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "f9c3dd3f-0b25-4c6b-88c1-f5e73213c6dc", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "418689008", + "display": "Allergy to grass pollen" + } + ], + "text": "Allergy to grass pollen" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:e8b91308-2205-42cd-8b50-0013a67f6c46", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "e8b91308-2205-42cd-8b50-0013a67f6c46", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "425525006", + "display": "Allergy to dairy product" + } + ], + "text": "Allergy to dairy product" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:bf642f63-7298-4026-97d7-e82b1f9e49af", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "bf642f63-7298-4026-97d7-e82b1f9e49af", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "417532002", + "display": "Allergy to fish" + } + ], + "text": "Allergy to fish" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:dbb6d6e6-43cf-4670-967d-35fe90d84775", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "dbb6d6e6-43cf-4670-967d-35fe90d84775", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "91934008", + "display": "Allergy to nut" + } + ], + "text": "Allergy to nut" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:8ef4a099-3c31-42e6-b45d-0e96b2a0545c", + "resource": { + "resourceType": "AllergyIntolerance", + "id": "8ef4a099-3c31-42e6-b45d-0e96b2a0545c", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "type": "allergy", + "category": [ + "food" + ], + "criticality": "low", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "91935009", + "display": "Allergy to peanuts" + } + ], + "text": "Allergy to peanuts" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "assertedDate": "2019-09-14T06:13:10-07:00" + }, + "request": { + "method": "POST", + "url": "AllergyIntolerance" + } + }, + { + "fullUrl": "urn:uuid:8fc57162-5928-44bb-bf29-072747744f61", + "resource": { + "resourceType": "Observation", + "id": "8fc57162-5928-44bb-bf29-072747744f61", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6206-7", + "display": "Peanut IgE Ab in Serum" + } + ], + "text": "Peanut IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 13.815, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6958c9b3-3224-49b5-9cbf-1b1c1562e347", + "resource": { + "resourceType": "Observation", + "id": "6958c9b3-3224-49b5-9cbf-1b1c1562e347", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6273-7", + "display": "Walnut IgE Ab in Serum" + } + ], + "text": "Walnut IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 60.513, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:00383c72-18fe-477b-af20-e8b8a2417ced", + "resource": { + "resourceType": "Observation", + "id": "00383c72-18fe-477b-af20-e8b8a2417ced", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6082-2", + "display": "Codfish IgE Ab in Serum" + } + ], + "text": "Codfish IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 54.222, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:722e972a-9927-4dfd-8fec-d7cc7b19f8da", + "resource": { + "resourceType": "Observation", + "id": "722e972a-9927-4dfd-8fec-d7cc7b19f8da", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6246-3", + "display": "Shrimp IgE Ab in Serum" + } + ], + "text": "Shrimp IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.23565, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1c3ee88-9020-4d6e-8e9d-5b409f0c75e9", + "resource": { + "resourceType": "Observation", + "id": "a1c3ee88-9020-4d6e-8e9d-5b409f0c75e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6276-0", + "display": "Wheat IgE Ab in Serum" + } + ], + "text": "Wheat IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.27006, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b17db831-9797-4135-8cee-1503c1bf0a49", + "resource": { + "resourceType": "Observation", + "id": "b17db831-9797-4135-8cee-1503c1bf0a49", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6106-9", + "display": "Egg white IgE Ab in Serum" + } + ], + "text": "Egg white IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.31736, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbab2cc0-16ce-40bb-9c7b-de6a68b9358b", + "resource": { + "resourceType": "Observation", + "id": "bbab2cc0-16ce-40bb-9c7b-de6a68b9358b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6248-9", + "display": "Soybean IgE Ab in Serum" + } + ], + "text": "Soybean IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.075051, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a8a64a5-5879-4754-99da-488f05e7bd47", + "resource": { + "resourceType": "Observation", + "id": "8a8a64a5-5879-4754-99da-488f05e7bd47", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "7258-7", + "display": "Cow milk IgE Ab in Serum" + } + ], + "text": "Cow milk IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 73.784, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7efeb353-ee73-4c2a-a5bf-ed5350793915", + "resource": { + "resourceType": "Observation", + "id": "7efeb353-ee73-4c2a-a5bf-ed5350793915", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6189-5", + "display": "White oak IgE Ab in Serum" + } + ], + "text": "White oak IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.25997, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e151b09-505a-44d8-a744-536e6446f6c1", + "resource": { + "resourceType": "Observation", + "id": "8e151b09-505a-44d8-a744-536e6446f6c1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6085-5", + "display": "Common Ragweed IgE Ab in Serum" + } + ], + "text": "Common Ragweed IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 13.391, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53005f80-c9c1-4a21-b7c3-1137cfdc69a4", + "resource": { + "resourceType": "Observation", + "id": "53005f80-c9c1-4a21-b7c3-1137cfdc69a4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6833-8", + "display": "Cat dander IgE Ab in Serum" + } + ], + "text": "Cat dander IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 49.410, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c91cad0-029d-407f-98aa-962fcca7b815", + "resource": { + "resourceType": "Observation", + "id": "2c91cad0-029d-407f-98aa-962fcca7b815", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6095-4", + "display": "American house dust mite IgE Ab in Serum" + } + ], + "text": "American house dust mite IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 56.759, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f7c0b965-36ee-480b-ad3d-4c1bc2951532", + "resource": { + "resourceType": "Observation", + "id": "f7c0b965-36ee-480b-ad3d-4c1bc2951532", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6075-6", + "display": "Cladosporium herbarum IgE Ab in Serum" + } + ], + "text": "Cladosporium herbarum IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 17.679, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b009f01-57a4-4836-849b-20cc6dce5386", + "resource": { + "resourceType": "Observation", + "id": "0b009f01-57a4-4836-849b-20cc6dce5386", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6844-5", + "display": "Honey bee IgE Ab in Serum" + } + ], + "text": "Honey bee IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.095614, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0a2e899-b32b-427e-8ccc-c74f584cf6f3", + "resource": { + "resourceType": "Observation", + "id": "b0a2e899-b32b-427e-8ccc-c74f584cf6f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6158-0", + "display": "Latex IgE Ab in Serum" + } + ], + "text": "Latex IgE Ab in Serum" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "effectiveDateTime": "2019-09-14T06:13:10-07:00", + "issued": "2019-09-14T06:13:10.107-07:00", + "valueQuantity": { + "value": 0.18124, + "unit": "kU/L", + "system": "http://unitsofmeasure.org", + "code": "kU/L" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:24928c23-eeda-4edc-8cef-b1e11de40565", + "resource": { + "resourceType": "Procedure", + "id": "24928c23-eeda-4edc-8cef-b1e11de40565", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "395142003", + "display": "Allergy screening test" + } + ], + "text": "Allergy screening test" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "performedPeriod": { + "start": "2019-09-14T06:13:10-07:00", + "end": "2019-09-14T06:34:10-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ab80d554-f980-4306-9c08-5b77069fa93b", + "resource": { + "resourceType": "MedicationRequest", + "id": "ab80d554-f980-4306-9c08-5b77069fa93b", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "997488", + "display": "Fexofenadine hydrochloride 30 MG Oral Tablet" + } + ], + "text": "Fexofenadine hydrochloride 30 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "authoredOn": "2019-09-14T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + }, + "onBehalfOf": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:cba2410d-16d2-43e2-a222-26f385e6724e", + "resource": { + "resourceType": "Claim", + "id": "cba2410d-16d2-43e2-a222-26f385e6724e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-09-14T06:13:10-07:00", + "end": "2019-09-14T06:49:10-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "prescription": { + "reference": "urn:uuid:ab80d554-f980-4306-9c08-5b77069fa93b" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + } + ] + } + ], + "total": { + "value": 20.24, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0327ac93-7e68-46c2-9283-6426749a334c", + "resource": { + "resourceType": "MedicationRequest", + "id": "0327ac93-7e68-46c2-9283-6426749a334c", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1870230", + "display": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" + } + ], + "text": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + }, + "authoredOn": "2019-09-14T06:13:10-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + }, + "onBehalfOf": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8fc44a50-9eb4-4614-9ec5-9f75cd32bbd5", + "resource": { + "resourceType": "Claim", + "id": "8fc44a50-9eb4-4614-9ec5-9f75cd32bbd5", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-09-14T06:13:10-07:00", + "end": "2019-09-14T06:49:10-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "prescription": { + "reference": "urn:uuid:0327ac93-7e68-46c2-9283-6426749a334c" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + } + ] + } + ], + "total": { + "value": 237.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:60a4837f-cb91-4a30-a85f-2d1a0b9252a7", + "resource": { + "resourceType": "Claim", + "id": "60a4837f-cb91-4a30-a85f-2d1a0b9252a7", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-09-14T06:13:10-07:00", + "end": "2019-09-14T06:49:10-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:24928c23-eeda-4edc-8cef-b1e11de40565" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "395142003" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:96694062-d875-40d7-b3a1-a0aa4f032f05", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "96694062-d875-40d7-b3a1-a0aa4f032f05", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "60a4837f-cb91-4a30-a85f-2d1a0b9252a7" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-09-14T06:49:10-07:00", + "end": "2020-09-14T06:49:10-07:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "395142003" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544", + "resource": { + "resourceType": "Encounter", + "id": "3353a56d-d257-49bc-9040-275c92819544", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" + } + } + ], + "period": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:43:10-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1e7d21ec-e1c9-45c9-97cc-40da922a294b", + "resource": { + "resourceType": "Observation", + "id": "1e7d21ec-e1c9-45c9-97cc-40da922a294b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 78.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3cce288-7f4f-4737-a519-5faf3ac2230c", + "resource": { + "resourceType": "Observation", + "id": "b3cce288-7f4f-4737-a519-5faf3ac2230c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49c20729-78ac-4475-8e09-b13b3eddcc6f", + "resource": { + "resourceType": "Observation", + "id": "49c20729-78ac-4475-8e09-b13b3eddcc6f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 10.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd79636f-977d-4736-afed-8da9b5bd4302", + "resource": { + "resourceType": "Observation", + "id": "cd79636f-977d-4736-afed-8da9b5bd4302", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 53.218, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4893f58d-6824-4e25-842f-37456823f56a", + "resource": { + "resourceType": "Observation", + "id": "4893f58d-6824-4e25-842f-37456823f56a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 46.310, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:147d2240-53f2-4cd9-86d8-bf240ce5c774", + "resource": { + "resourceType": "Observation", + "id": "147d2240-53f2-4cd9-86d8-bf240ce5c774", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 84, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 131, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1a40cb7-a9ec-4426-9537-73786a3a0653", + "resource": { + "resourceType": "Observation", + "id": "f1a40cb7-a9ec-4426-9537-73786a3a0653", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 97, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fede4f58-46f7-4431-9c41-7ec51ac7e374", + "resource": { + "resourceType": "Observation", + "id": "fede4f58-46f7-4431-9c41-7ec51ac7e374", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f6361e81-ce8f-41b7-b912-c94e72128480", + "resource": { + "resourceType": "Observation", + "id": "f6361e81-ce8f-41b7-b912-c94e72128480", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "effectiveDateTime": "2019-12-09T05:13:10-08:00", + "issued": "2019-12-09T05:13:10.107-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87", + "resource": { + "resourceType": "Procedure", + "id": "f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "performedPeriod": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:28:10-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ba32549f-f9e9-4274-bc4f-9a914f0a8991", + "resource": { + "resourceType": "Immunization", + "id": "ba32549f-f9e9-4274-bc4f-9a914f0a8991", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "encounter": { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + }, + "date": "2019-12-09T05:13:10-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:5bb3d5cb-571d-474d-acd7-f18500b08680", + "resource": { + "resourceType": "Claim", + "id": "5bb3d5cb-571d-474d-acd7-f18500b08680", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-12-09T05:13:10-08:00", + "end": "2019-12-09T05:43:10-08:00" + }, + "organization": { + "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ba32549f-f9e9-4274-bc4f-9a914f0a8991" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 318.06, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f17b038c-4245-4a8a-a6cd-4e9d5df76067", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f17b038c-4245-4a8a-a6cd-4e9d5df76067", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Humana" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5bb3d5cb-571d-474d-acd7-f18500b08680" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-12-09T05:43:10-08:00", + "end": "2020-12-09T05:43:10-08:00" + }, + "provider": { + "identifier": { + "value": "d726e757-2975-3d6a-bdf9-517546432499" + } + }, + "organization": { + "identifier": { + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 63.612, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 254.448, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 318.06, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 318.06, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 366.86400000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494", + "resource": { + "resourceType": "Encounter", + "id": "835db05f-836f-4323-bb98-3f330070e494", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2019-12-24T05:13:10-08:00", + "end": "2019-12-24T05:28:10-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7", + "resource": { + "resourceType": "Condition", + "id": "b2631fd3-7099-4f20-b63b-9f6ecc79dec7", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" + }, + "onsetDateTime": "2019-12-24T05:13:10-08:00", + "assertedDate": "2019-12-24T05:13:10-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:d0e6004d-b188-495e-aa52-f68c7681a924", + "resource": { + "resourceType": "MedicationRequest", + "id": "d0e6004d-b188-495e-aa52-f68c7681a924", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "context": { + "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" + }, + "authoredOn": "2019-12-24T05:13:10-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d7530091-8f70-4522-9a9b-df856caacb40", + "resource": { + "resourceType": "Claim", + "id": "d7530091-8f70-4522-9a9b-df856caacb40", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-12-24T05:13:10-08:00", + "end": "2019-12-24T05:28:10-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:d0e6004d-b188-495e-aa52-f68c7681a924" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" + } + ] + } + ], + "total": { + "value": 4.98, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e322a7e0-7755-4420-8d76-e956d8ef64bb", + "resource": { + "resourceType": "Claim", + "id": "e322a7e0-7755-4420-8d76-e956d8ef64bb", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-12-24T05:13:10-08:00", + "end": "2019-12-24T05:28:10-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c0afc820-908b-4b15-a534-d05e037ca605", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c0afc820-908b-4b15-a534-d05e037ca605", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e322a7e0-7755-4420-8d76-e956d8ef64bb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" + }, + "billablePeriod": { + "start": "2019-12-24T05:28:10-08:00", + "end": "2020-12-24T05:28:10-08:00" + }, + "created": "2019-12-24T05:28:10-08:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:e322a7e0-7755-4420-8d76-e956d8ef64bb" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-12-24T05:13:10-08:00", + "end": "2019-12-24T05:28:10-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-12-24T05:13:10-08:00", + "end": "2019-12-24T05:28:10-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json new file mode 100644 index 000000000000..1f4a6a88e235 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json @@ -0,0 +1,22688 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c", + "resource": { + "resourceType": "Patient", + "id": "df27a976-5c5e-4b84-ad00-fe32972dce9c", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -4106806494818543385 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2106-3", + "display": "White" + } + }, + { + "url": "text", + "valueString": "White" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Babette571 Dooley940" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Needham", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.003367159185079892 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 5.99663284081492 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-61-1293" + } + ], + "name": [ + { + "use": "official", + "family": "Lang846", + "given": [ + "Cris921" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-281-6453", + "use": "home" + } + ], + "gender": "female", + "birthDate": "2013-11-29", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.09755452320718 + }, + { + "url": "longitude", + "valueDecimal": -70.83495946935349 + } + ] + } + ], + "line": [ + "1082 Vandervort Avenue Apt 70" + ], + "city": "Hanover", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5", + "resource": { + "resourceType": "Organization", + "id": "97066a41-4fd5-3d4e-b382-a978fb20a5d5", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP33234", + "telecom": [ + { + "system": "phone", + "value": "781-829-9300" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.122956 + }, + { + "url": "longitude", + "valueDecimal": -70.85631 + } + ] + } + ], + "line": [ + "198 COLUMBIA RD" + ], + "city": "HANOVER", + "state": "MA", + "postalCode": "02339-2380", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553", + "resource": { + "resourceType": "Practitioner", + "id": "f9f09686-21e4-3b27-ba68-9b7313d9a553", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "22570" + } + ], + "active": true, + "name": [ + { + "family": "Homenick806", + "given": [ + "Evan94" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "198 COLUMBIA RD" + ], + "city": "HANOVER", + "state": "MA", + "postalCode": "02339-2380", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0", + "resource": { + "resourceType": "Encounter", + "id": "de64ec54-276c-4287-b6c1-3584fee7f8b0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2013-11-29T15:21:32-08:00", + "end": "2013-11-29T15:51:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1c1addf3-8cd7-4541-bf69-fce5e21f5346", + "resource": { + "resourceType": "Observation", + "id": "1c1addf3-8cd7-4541-bf69-fce5e21f5346", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 49.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bb059a3a-5c99-48ba-b620-ba10f8f98816", + "resource": { + "resourceType": "Observation", + "id": "bb059a3a-5c99-48ba-b620-ba10f8f98816", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ef19301-2656-4de6-bc0b-40543e493be5", + "resource": { + "resourceType": "Observation", + "id": "0ef19301-2656-4de6-bc0b-40543e493be5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 3.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9a0ddc69-b028-4339-b4a3-a1d4894e774e", + "resource": { + "resourceType": "Observation", + "id": "9a0ddc69-b028-4339-b4a3-a1d4894e774e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 20.501, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc528b99-0633-478b-98ab-6fa2b9d8cd38", + "resource": { + "resourceType": "Observation", + "id": "fc528b99-0633-478b-98ab-6fa2b9d8cd38", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 33.290, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f87afcc1-9ebb-43ac-bd25-0912da6cb3e5", + "resource": { + "resourceType": "Observation", + "id": "f87afcc1-9ebb-43ac-bd25-0912da6cb3e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 128, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34f7c63f-d72a-495a-b640-b3825a84efb1", + "resource": { + "resourceType": "Observation", + "id": "34f7c63f-d72a-495a-b640-b3825a84efb1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:788cf18f-9794-4caa-ab3a-5b6531b8f1ad", + "resource": { + "resourceType": "Observation", + "id": "788cf18f-9794-4caa-ab3a-5b6531b8f1ad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8d773eb-40fc-406c-aea2-02d67e49d897", + "resource": { + "resourceType": "Observation", + "id": "e8d773eb-40fc-406c-aea2-02d67e49d897", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 5.1116, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:94c4633e-60b3-452e-b6e8-0fee0c7a806c", + "resource": { + "resourceType": "Observation", + "id": "94c4633e-60b3-452e-b6e8-0fee0c7a806c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 4.0192, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b699973-0a40-4c98-b8f3-3a5b8823fdb5", + "resource": { + "resourceType": "Observation", + "id": "4b699973-0a40-4c98-b8f3-3a5b8823fdb5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 16.471, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bccd98eb-ded2-409e-911f-a2640ebf6661", + "resource": { + "resourceType": "Observation", + "id": "bccd98eb-ded2-409e-911f-a2640ebf6661", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 41.561, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:75f18ec8-d50a-4489-a1ba-f668473aadf7", + "resource": { + "resourceType": "Observation", + "id": "75f18ec8-d50a-4489-a1ba-f668473aadf7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 86.346, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6c4f772b-2473-49ea-a755-ed2b2b32f570", + "resource": { + "resourceType": "Observation", + "id": "6c4f772b-2473-49ea-a755-ed2b2b32f570", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 30.141, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6f3b27fb-f097-47b9-a24d-6b768568ca24", + "resource": { + "resourceType": "Observation", + "id": "6f3b27fb-f097-47b9-a24d-6b768568ca24", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 35.300, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", + "resource": { + "resourceType": "Observation", + "id": "38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 43.444, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66bc2209-0855-4cda-9dd4-6787afcb0cd0", + "resource": { + "resourceType": "Observation", + "id": "66bc2209-0855-4cda-9dd4-6787afcb0cd0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 442.51, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", + "resource": { + "resourceType": "Observation", + "id": "f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 190.14, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d74fd07e-7aa0-4fc4-9f16-11e8fb682119", + "resource": { + "resourceType": "Observation", + "id": "d74fd07e-7aa0-4fc4-9f16-11e8fb682119", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueQuantity": { + "value": 11.668, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ace82b1-3297-43ab-bef9-a14d931f2837", + "resource": { + "resourceType": "Observation", + "id": "2ace82b1-3297-43ab-bef9-a14d931f2837", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fec49be6-9b1c-44b2-bdf4-1d1f6c884666", + "resource": { + "resourceType": "Procedure", + "id": "fec49be6-9b1c-44b2-bdf4-1d1f6c884666", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "performedPeriod": { + "start": "2013-11-29T15:21:32-08:00", + "end": "2013-11-29T15:36:32-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7257de06-fe3c-462b-8a68-f0de095971e8", + "resource": { + "resourceType": "Immunization", + "id": "7257de06-fe3c-462b-8a68-f0de095971e8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "date": "2013-11-29T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:37349538-615a-43b2-a2c8-23bb185da500", + "resource": { + "resourceType": "DiagnosticReport", + "id": "37349538-615a-43b2-a2c8-23bb185da500", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + }, + "effectiveDateTime": "2013-11-29T15:21:32-08:00", + "issued": "2013-11-29T15:21:32.447-08:00", + "result": [ + { + "reference": "urn:uuid:e8d773eb-40fc-406c-aea2-02d67e49d897", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:94c4633e-60b3-452e-b6e8-0fee0c7a806c", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:4b699973-0a40-4c98-b8f3-3a5b8823fdb5", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:bccd98eb-ded2-409e-911f-a2640ebf6661", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:75f18ec8-d50a-4489-a1ba-f668473aadf7", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:6c4f772b-2473-49ea-a755-ed2b2b32f570", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:6f3b27fb-f097-47b9-a24d-6b768568ca24", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:66bc2209-0855-4cda-9dd4-6787afcb0cd0", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:d74fd07e-7aa0-4fc4-9f16-11e8fb682119", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:cc6b7dd9-633d-404b-a49d-302d27125fdd", + "resource": { + "resourceType": "Claim", + "id": "cc6b7dd9-633d-404b-a49d-302d27125fdd", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2013-11-29T15:21:32-08:00", + "end": "2013-11-29T15:51:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7257de06-fe3c-462b-8a68-f0de095971e8" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:fec49be6-9b1c-44b2-bdf4-1d1f6c884666" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 396.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d5243537-4f20-4fb4-98e9-60f75b3c2522", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d5243537-4f20-4fb4-98e9-60f75b3c2522", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cc6b7dd9-633d-404b-a49d-302d27125fdd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-11-29T15:51:32-08:00", + "end": "2014-11-29T15:51:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 79.278, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 317.112, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 396.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 396.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 429.528, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4", + "resource": { + "resourceType": "Encounter", + "id": "28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2014-01-03T15:21:32-08:00", + "end": "2014-01-03T15:51:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2961e432-2b46-4377-8c9b-7da30371f351", + "resource": { + "resourceType": "Observation", + "id": "2961e432-2b46-4377-8c9b-7da30371f351", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 53.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31f23f64-a40b-4821-924d-ed9a8dfb0e79", + "resource": { + "resourceType": "Observation", + "id": "31f23f64-a40b-4821-924d-ed9a8dfb0e79", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d212de82-b1e4-47d4-add4-b90ef5e03379", + "resource": { + "resourceType": "Observation", + "id": "d212de82-b1e4-47d4-add4-b90ef5e03379", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 3.9000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:231d1996-058c-4fdc-a469-66633a7313bc", + "resource": { + "resourceType": "Observation", + "id": "231d1996-058c-4fdc-a469-66633a7313bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 21.188, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9613a409-4a15-4d53-b41a-10c35ef6bcec", + "resource": { + "resourceType": "Observation", + "id": "9613a409-4a15-4d53-b41a-10c35ef6bcec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 36.710, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3928d564-52b0-4c0f-adce-536c8061b234", + "resource": { + "resourceType": "Observation", + "id": "3928d564-52b0-4c0f-adce-536c8061b234", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 87, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7590e440-193b-42ef-8d4c-71d29923bf22", + "resource": { + "resourceType": "Observation", + "id": "7590e440-193b-42ef-8d4c-71d29923bf22", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 75, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f728cf6-7135-44fe-b9cb-0e63343e32e6", + "resource": { + "resourceType": "Observation", + "id": "1f728cf6-7135-44fe-b9cb-0e63343e32e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:05bda50c-3e77-4aa4-b854-5f411685578f", + "resource": { + "resourceType": "Observation", + "id": "05bda50c-3e77-4aa4-b854-5f411685578f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "effectiveDateTime": "2014-01-03T15:21:32-08:00", + "issued": "2014-01-03T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a58560af-b5ab-48b5-b465-59d9736c8a21", + "resource": { + "resourceType": "Procedure", + "id": "a58560af-b5ab-48b5-b465-59d9736c8a21", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "performedPeriod": { + "start": "2014-01-03T15:21:32-08:00", + "end": "2014-01-03T15:36:32-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:82baa6f9-c1d0-4db2-9d46-07d92575974a", + "resource": { + "resourceType": "Immunization", + "id": "82baa6f9-c1d0-4db2-9d46-07d92575974a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + }, + "date": "2014-01-03T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7b398516-ac55-4ca7-9beb-3cd314e23d28", + "resource": { + "resourceType": "Claim", + "id": "7b398516-ac55-4ca7-9beb-3cd314e23d28", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-01-03T15:21:32-08:00", + "end": "2014-01-03T15:51:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:82baa6f9-c1d0-4db2-9d46-07d92575974a" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a58560af-b5ab-48b5-b465-59d9736c8a21" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 648.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:03311493-acff-48c7-9cdc-736fc416bfb6", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "03311493-acff-48c7-9cdc-736fc416bfb6", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7b398516-ac55-4ca7-9beb-3cd314e23d28" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-01-03T15:51:32-08:00", + "end": "2015-01-03T15:51:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 129.70000000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 518.8000000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 648.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 648.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 631.2160000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567", + "resource": { + "resourceType": "Encounter", + "id": "abda8627-8b93-49d9-ac98-e7623872c567", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2014-03-07T15:21:32-08:00", + "end": "2014-03-07T15:36:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7416c609-f9e2-4c37-80b5-e87bd022abec", + "resource": { + "resourceType": "Observation", + "id": "7416c609-f9e2-4c37-80b5-e87bd022abec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 58.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a796ce16-4a5e-4d3e-bfd8-af3431b8b760", + "resource": { + "resourceType": "Observation", + "id": "a796ce16-4a5e-4d3e-bfd8-af3431b8b760", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b7b94a8-db76-492a-8b40-2617a2d0d69c", + "resource": { + "resourceType": "Observation", + "id": "4b7b94a8-db76-492a-8b40-2617a2d0d69c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 5.1000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:70e80a37-4ca2-40ce-a530-3cac6c0ef202", + "resource": { + "resourceType": "Observation", + "id": "70e80a37-4ca2-40ce-a530-3cac6c0ef202", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 22.579, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fecdbf07-2ca3-48be-95e4-a1e25abe3bce", + "resource": { + "resourceType": "Observation", + "id": "fecdbf07-2ca3-48be-95e4-a1e25abe3bce", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 39.290, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db07f377-0b5d-4174-803e-f1801c801e7a", + "resource": { + "resourceType": "Observation", + "id": "db07f377-0b5d-4174-803e-f1801c801e7a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e9bfc74c-7285-4bbc-a92f-ca4739fb7779", + "resource": { + "resourceType": "Observation", + "id": "e9bfc74c-7285-4bbc-a92f-ca4739fb7779", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 64, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35580509-4731-42b6-8c25-61ec3485bf83", + "resource": { + "resourceType": "Observation", + "id": "35580509-4731-42b6-8c25-61ec3485bf83", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac3a0166-cd40-41b3-aae4-1b57e90eb5ed", + "resource": { + "resourceType": "Observation", + "id": "ac3a0166-cd40-41b3-aae4-1b57e90eb5ed", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "effectiveDateTime": "2014-03-07T15:21:32-08:00", + "issued": "2014-03-07T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f6b147df-1285-44f2-9a30-b9ec95f8f820", + "resource": { + "resourceType": "Immunization", + "id": "f6b147df-1285-44f2-9a30-b9ec95f8f820", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "date": "2014-03-07T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:04354efd-8ddb-4d5d-a51e-f729a1c64c6b", + "resource": { + "resourceType": "Immunization", + "id": "04354efd-8ddb-4d5d-a51e-f729a1c64c6b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "date": "2014-03-07T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:91191980-ecd5-4e81-b1bd-819fdff84f37", + "resource": { + "resourceType": "Immunization", + "id": "91191980-ecd5-4e81-b1bd-819fdff84f37", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "date": "2014-03-07T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:966ccb19-d147-47f9-a369-c77bf8cc96cc", + "resource": { + "resourceType": "Immunization", + "id": "966ccb19-d147-47f9-a369-c77bf8cc96cc", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "date": "2014-03-07T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:dc0b0d2a-9fda-4df8-baf7-20c14b644237", + "resource": { + "resourceType": "Immunization", + "id": "dc0b0d2a-9fda-4df8-baf7-20c14b644237", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + }, + "date": "2014-03-07T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:91ec0186-757e-4933-a46e-be9fb938bd43", + "resource": { + "resourceType": "Claim", + "id": "91ec0186-757e-4933-a46e-be9fb938bd43", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-03-07T15:21:32-08:00", + "end": "2014-03-07T15:36:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f6b147df-1285-44f2-9a30-b9ec95f8f820" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:04354efd-8ddb-4d5d-a51e-f729a1c64c6b" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:91191980-ecd5-4e81-b1bd-819fdff84f37" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:966ccb19-d147-47f9-a369-c77bf8cc96cc" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:dc0b0d2a-9fda-4df8-baf7-20c14b644237" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dce04e16-40c4-4f17-87f7-40a3b12ec920", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "dce04e16-40c4-4f17-87f7-40a3b12ec920", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "91ec0186-757e-4933-a46e-be9fb938bd43" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-03-07T15:36:32-08:00", + "end": "2015-03-07T15:36:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad", + "resource": { + "resourceType": "Encounter", + "id": "ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2014-05-09T16:21:32-07:00", + "end": "2014-05-09T16:51:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:dd43d232-e6f0-4c33-8844-db440c3ca4c8", + "resource": { + "resourceType": "Observation", + "id": "dd43d232-e6f0-4c33-8844-db440c3ca4c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 62.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f128b0e5-15c5-4828-8754-ca1999bd746f", + "resource": { + "resourceType": "Observation", + "id": "f128b0e5-15c5-4828-8754-ca1999bd746f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7f66a421-5ccf-4573-82f4-0878643e39fe", + "resource": { + "resourceType": "Observation", + "id": "7f66a421-5ccf-4573-82f4-0878643e39fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 6.1000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e53a668-41f2-4af9-9d99-46c565fd2e2e", + "resource": { + "resourceType": "Observation", + "id": "4e53a668-41f2-4af9-9d99-46c565fd2e2e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 25.754, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c11f2f6-7f75-4e2e-9399-1539de7b5c29", + "resource": { + "resourceType": "Observation", + "id": "2c11f2f6-7f75-4e2e-9399-1539de7b5c29", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 40.940, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:95cbf501-33fb-4a35-b86a-8e35f814b17d", + "resource": { + "resourceType": "Observation", + "id": "95cbf501-33fb-4a35-b86a-8e35f814b17d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb516dbb-be5f-4c25-8e3b-a717c7cd45dd", + "resource": { + "resourceType": "Observation", + "id": "cb516dbb-be5f-4c25-8e3b-a717c7cd45dd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:305c80be-919b-4b6a-8727-39d92620ad0f", + "resource": { + "resourceType": "Observation", + "id": "305c80be-919b-4b6a-8727-39d92620ad0f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49a7bae4-0919-4112-9c0f-83925b2d902c", + "resource": { + "resourceType": "Observation", + "id": "49a7bae4-0919-4112-9c0f-83925b2d902c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "effectiveDateTime": "2014-05-09T16:21:32-07:00", + "issued": "2014-05-09T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e4a2c1d-813e-4206-b353-00ece4b65402", + "resource": { + "resourceType": "Procedure", + "id": "5e4a2c1d-813e-4206-b353-00ece4b65402", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "performedPeriod": { + "start": "2014-05-09T16:21:32-07:00", + "end": "2014-05-09T16:36:32-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34", + "resource": { + "resourceType": "Immunization", + "id": "7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "date": "2014-05-09T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1", + "resource": { + "resourceType": "Immunization", + "id": "2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "date": "2014-05-09T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b80bd7af-769c-4b52-8290-3a6ed69f545c", + "resource": { + "resourceType": "Immunization", + "id": "b80bd7af-769c-4b52-8290-3a6ed69f545c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "date": "2014-05-09T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c9a3c887-4bff-4442-b081-db928259ef2a", + "resource": { + "resourceType": "Immunization", + "id": "c9a3c887-4bff-4442-b081-db928259ef2a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "date": "2014-05-09T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:45f69ce9-e06d-425a-a90b-927cb5d34e6d", + "resource": { + "resourceType": "Immunization", + "id": "45f69ce9-e06d-425a-a90b-927cb5d34e6d", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + }, + "date": "2014-05-09T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e5381efa-2d2e-441b-985c-faebe470b15a", + "resource": { + "resourceType": "Claim", + "id": "e5381efa-2d2e-441b-985c-faebe470b15a", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-05-09T16:21:32-07:00", + "end": "2014-05-09T16:51:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:b80bd7af-769c-4b52-8290-3a6ed69f545c" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c9a3c887-4bff-4442-b081-db928259ef2a" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:45f69ce9-e06d-425a-a90b-927cb5d34e6d" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:5e4a2c1d-813e-4206-b353-00ece4b65402" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 831.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3a5031aa-765e-4ec0-a6e6-12ef90c18dc8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3a5031aa-765e-4ec0-a6e6-12ef90c18dc8", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e5381efa-2d2e-441b-985c-faebe470b15a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-05-09T16:51:32-07:00", + "end": "2015-05-09T16:51:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 166.286, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 665.144, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 831.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 831.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1227.2240000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b", + "resource": { + "resourceType": "Encounter", + "id": "43174bce-2315-4d09-a84c-953ed0d9385b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2014-08-08T16:21:32-07:00", + "end": "2014-08-08T16:51:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:362a225c-da34-43d9-926f-e4105eb3dfb5", + "resource": { + "resourceType": "Observation", + "id": "362a225c-da34-43d9-926f-e4105eb3dfb5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 66.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25c13fd9-fc4f-4e0a-9228-45bb8e4a12e4", + "resource": { + "resourceType": "Observation", + "id": "25c13fd9-fc4f-4e0a-9228-45bb8e4a12e4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:16c09d37-af4b-43cc-af83-03edcace4551", + "resource": { + "resourceType": "Observation", + "id": "16c09d37-af4b-43cc-af83-03edcace4551", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 7.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fd997161-2225-4582-8b69-8a349b78026a", + "resource": { + "resourceType": "Observation", + "id": "fd997161-2225-4582-8b69-8a349b78026a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 32.896, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3694ae02-1942-4900-9042-99510a918435", + "resource": { + "resourceType": "Observation", + "id": "3694ae02-1942-4900-9042-99510a918435", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 42.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba618b9a-a15a-4067-ae4d-1301aacf133a", + "resource": { + "resourceType": "Observation", + "id": "ba618b9a-a15a-4067-ae4d-1301aacf133a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33229ce8-48fd-447f-ab2f-a6ba8da654fd", + "resource": { + "resourceType": "Observation", + "id": "33229ce8-48fd-447f-ab2f-a6ba8da654fd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 96, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b7d305e-f91e-4200-97c9-1b8c6e203f0d", + "resource": { + "resourceType": "Observation", + "id": "4b7d305e-f91e-4200-97c9-1b8c6e203f0d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f74d9df1-af18-43c1-b6a5-e15a827ebbdc", + "resource": { + "resourceType": "Observation", + "id": "f74d9df1-af18-43c1-b6a5-e15a827ebbdc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "effectiveDateTime": "2014-08-08T16:21:32-07:00", + "issued": "2014-08-08T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9", + "resource": { + "resourceType": "Procedure", + "id": "2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "performedPeriod": { + "start": "2014-08-08T16:21:32-07:00", + "end": "2014-08-08T16:36:32-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ed53940a-e1f5-46f3-ac61-de1af1fbab7c", + "resource": { + "resourceType": "Immunization", + "id": "ed53940a-e1f5-46f3-ac61-de1af1fbab7c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "date": "2014-08-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:74132144-c817-4926-834b-20e10eacd426", + "resource": { + "resourceType": "Immunization", + "id": "74132144-c817-4926-834b-20e10eacd426", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "date": "2014-08-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:659b9e5b-365e-415f-bc81-be05f53f23ec", + "resource": { + "resourceType": "Immunization", + "id": "659b9e5b-365e-415f-bc81-be05f53f23ec", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "date": "2014-08-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:180a9ca7-a4b3-4215-b689-395169136224", + "resource": { + "resourceType": "Immunization", + "id": "180a9ca7-a4b3-4215-b689-395169136224", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "date": "2014-08-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:141770a5-bd98-4aa9-8e74-6971b14ade5a", + "resource": { + "resourceType": "Immunization", + "id": "141770a5-bd98-4aa9-8e74-6971b14ade5a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + }, + "date": "2014-08-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:dae75b9c-3d5b-463f-baa0-2e33998b103d", + "resource": { + "resourceType": "Claim", + "id": "dae75b9c-3d5b-463f-baa0-2e33998b103d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-08-08T16:21:32-07:00", + "end": "2014-08-08T16:51:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ed53940a-e1f5-46f3-ac61-de1af1fbab7c" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:74132144-c817-4926-834b-20e10eacd426" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:659b9e5b-365e-415f-bc81-be05f53f23ec" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:180a9ca7-a4b3-4215-b689-395169136224" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:141770a5-bd98-4aa9-8e74-6971b14ade5a" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 732.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:00be94df-3f1d-454c-bd48-31e39cdd0544", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "00be94df-3f1d-454c-bd48-31e39cdd0544", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "dae75b9c-3d5b-463f-baa0-2e33998b103d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-08-08T16:51:32-07:00", + "end": "2015-08-08T16:51:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 146.504, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 586.016, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 732.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 732.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1148.096, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5", + "resource": { + "resourceType": "Encounter", + "id": "a62ab6f4-b2c3-4bec-ae76-9213db8c54d5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2014-11-07T15:21:32-08:00", + "end": "2014-11-07T15:51:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:be670ff4-78cb-4000-8935-371664a36a5c", + "resource": { + "resourceType": "Observation", + "id": "be670ff4-78cb-4000-8935-371664a36a5c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 70.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:93402cf4-52b0-476f-8b1d-5ed86d07022d", + "resource": { + "resourceType": "Observation", + "id": "93402cf4-52b0-476f-8b1d-5ed86d07022d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:055519a3-6d32-40de-8935-d3cb3b24d9ae", + "resource": { + "resourceType": "Observation", + "id": "055519a3-6d32-40de-8935-d3cb3b24d9ae", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 8.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d5198a4b-c61c-4571-9146-0b99de769332", + "resource": { + "resourceType": "Observation", + "id": "d5198a4b-c61c-4571-9146-0b99de769332", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 36.840, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7aa4496-f8e4-49aa-9ae6-e490e290954e", + "resource": { + "resourceType": "Observation", + "id": "b7aa4496-f8e4-49aa-9ae6-e490e290954e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 43.730, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d463a6c7-90b5-4204-8b58-48e60342fe24", + "resource": { + "resourceType": "Observation", + "id": "d463a6c7-90b5-4204-8b58-48e60342fe24", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 73, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 131, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c0b7b28e-2a6a-4ba7-a031-8036203855bd", + "resource": { + "resourceType": "Observation", + "id": "c0b7b28e-2a6a-4ba7-a031-8036203855bd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 77, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c2a4b1a0-3709-4020-aeb8-9d8acc017ffb", + "resource": { + "resourceType": "Observation", + "id": "c2a4b1a0-3709-4020-aeb8-9d8acc017ffb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a05af24-5448-4ac4-9c7e-e99eb7f0d202", + "resource": { + "resourceType": "Observation", + "id": "5a05af24-5448-4ac4-9c7e-e99eb7f0d202", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "effectiveDateTime": "2014-11-07T15:21:32-08:00", + "issued": "2014-11-07T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6295f137-97aa-4765-803e-1680110779dc", + "resource": { + "resourceType": "Procedure", + "id": "6295f137-97aa-4765-803e-1680110779dc", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + }, + "performedPeriod": { + "start": "2014-11-07T15:21:32-08:00", + "end": "2014-11-07T15:36:32-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3d05bc39-e280-4dab-8a71-3f5820714393", + "resource": { + "resourceType": "Claim", + "id": "3d05bc39-e280-4dab-8a71-3f5820714393", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-11-07T15:21:32-08:00", + "end": "2014-11-07T15:51:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:6295f137-97aa-4765-803e-1680110779dc" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 702.98, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a476e888-3e52-4220-abbb-b29d018c6af6", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a476e888-3e52-4220-abbb-b29d018c6af6", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3d05bc39-e280-4dab-8a71-3f5820714393" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-11-07T15:51:32-08:00", + "end": "2015-11-07T15:51:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 140.596, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 562.384, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 702.98, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 702.98, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.384, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", + "resource": { + "resourceType": "Organization", + "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "GOOD SAMARITAN MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5084273000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "resource": { + "resourceType": "Practitioner", + "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "460" + } + ], + "active": true, + "name": [ + { + "family": "Fay398", + "given": [ + "Deneen201" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613", + "resource": { + "resourceType": "Encounter", + "id": "5daa0556-37d5-4c15-af85-f14841b26613", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2014-11-26T15:21:32-08:00", + "end": "2014-11-26T15:42:32-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796", + "resource": { + "resourceType": "Condition", + "id": "ad242a39-266b-4164-91ca-232fd24a7796", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + }, + "onsetDateTime": "2014-11-26T15:21:32-08:00", + "abatementDateTime": "2014-12-10T15:21:32-08:00", + "assertedDate": "2014-11-26T15:21:32-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:e77d1595-3949-4c37-875f-f1cbec1b8aa5", + "resource": { + "resourceType": "Procedure", + "id": "e77d1595-3949-4c37-875f-f1cbec1b8aa5", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "269911007", + "display": "Sputum examination (procedure)" + } + ], + "text": "Sputum examination (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + }, + "performedPeriod": { + "start": "2014-11-26T15:21:32-08:00", + "end": "2014-11-26T15:27:32-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796", + "display": "Acute bronchitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:164b333c-2481-44f9-8c82-cbd92b873de8", + "resource": { + "resourceType": "MedicationRequest", + "id": "164b333c-2481-44f9-8c82-cbd92b873de8", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + }, + "authoredOn": "2014-11-26T15:21:32-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + }, + "onBehalfOf": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d8df90cb-d960-45d9-b2d0-8dd468394021", + "resource": { + "resourceType": "Claim", + "id": "d8df90cb-d960-45d9-b2d0-8dd468394021", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-11-26T15:21:32-08:00", + "end": "2014-11-26T15:42:32-08:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "prescription": { + "reference": "urn:uuid:164b333c-2481-44f9-8c82-cbd92b873de8" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + } + ] + } + ], + "total": { + "value": 5.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d42bf95c-51de-42e6-9f80-17621516592d", + "resource": { + "resourceType": "CarePlan", + "id": "d42bf95c-51de-42e6-9f80-17621516592d", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + }, + "period": { + "start": "2014-11-26T15:21:32-08:00", + "end": "2015-02-06T15:21:32-08:00" + }, + "addresses": [ + { + "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:cbbe8272-8363-4243-bc2b-511901299496", + "resource": { + "resourceType": "Claim", + "id": "cbbe8272-8363-4243-bc2b-511901299496", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-11-26T15:21:32-08:00", + "end": "2014-11-26T15:42:32-08:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e77d1595-3949-4c37-875f-f1cbec1b8aa5" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "269911007" + } + ] + }, + "net": { + "value": 5483.95, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:aab3e329-346e-4c87-a029-b3f11d14840a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "aab3e329-346e-4c87-a029-b3f11d14840a", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "cbbe8272-8363-4243-bc2b-511901299496" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-11-26T15:42:32-08:00", + "end": "2015-11-26T15:42:32-08:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "269911007" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 1096.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 4387.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 5483.95, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 5483.95, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 4387.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", + "resource": { + "resourceType": "Organization", + "id": "c44f361c-2efb-3050-8f97-0354a12e2920", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "c44f361c-2efb-3050-8f97-0354a12e2920" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5089417000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", + "resource": { + "resourceType": "Practitioner", + "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "200" + } + ], + "active": true, + "name": [ + { + "family": "Padberg411", + "given": [ + "Nila48" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb", + "resource": { + "resourceType": "Encounter", + "id": "cf852eb0-483e-428d-9271-486e9298b0fb", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" + } + } + ], + "period": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e4025922-c623-481b-a3c7-4e7947dce52d", + "resource": { + "resourceType": "Observation", + "id": "e4025922-c623-481b-a3c7-4e7947dce52d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 71.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:98a2275b-6b81-44c0-93bd-4d3bcf5bcb39", + "resource": { + "resourceType": "Observation", + "id": "98a2275b-6b81-44c0-93bd-4d3bcf5bcb39", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e57d90c2-71be-4c09-8a2a-77782b41a557", + "resource": { + "resourceType": "Observation", + "id": "e57d90c2-71be-4c09-8a2a-77782b41a557", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 8.7000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0651705c-6d2a-40b6-877f-e1d7492bade3", + "resource": { + "resourceType": "Observation", + "id": "0651705c-6d2a-40b6-877f-e1d7492bade3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 40.627, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32ce8504-bc8c-4e60-87ef-a91f35e51f1d", + "resource": { + "resourceType": "Observation", + "id": "32ce8504-bc8c-4e60-87ef-a91f35e51f1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 44.040, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36196780-3273-43f8-9404-13184bc36ad5", + "resource": { + "resourceType": "Observation", + "id": "36196780-3273-43f8-9404-13184bc36ad5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 118, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f531201b-6789-4db0-9e70-0371f37c01e7", + "resource": { + "resourceType": "Observation", + "id": "f531201b-6789-4db0-9e70-0371f37c01e7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c036e7c5-bc6b-4afb-abea-077f0287a423", + "resource": { + "resourceType": "Observation", + "id": "c036e7c5-bc6b-4afb-abea-077f0287a423", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d4540e5-371d-4d77-a378-1d8dd891a963", + "resource": { + "resourceType": "Observation", + "id": "6d4540e5-371d-4d77-a378-1d8dd891a963", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "effectiveDateTime": "2014-12-05T15:21:32-08:00", + "issued": "2014-12-05T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9dce2503-5576-49bc-b0df-a1d98ad4bc11", + "resource": { + "resourceType": "Immunization", + "id": "9dce2503-5576-49bc-b0df-a1d98ad4bc11", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "date": "2014-12-05T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007", + "resource": { + "resourceType": "Immunization", + "id": "e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "date": "2014-12-05T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:664cd96e-79b4-459c-8ba2-85d25bf84108", + "resource": { + "resourceType": "Immunization", + "id": "664cd96e-79b4-459c-8ba2-85d25bf84108", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "date": "2014-12-05T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a3d039d6-dd93-4d43-a741-6be2ee312375", + "resource": { + "resourceType": "Immunization", + "id": "a3d039d6-dd93-4d43-a741-6be2ee312375", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "date": "2014-12-05T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:81205a66-b680-4861-b1de-eb0ba1881202", + "resource": { + "resourceType": "Immunization", + "id": "81205a66-b680-4861-b1de-eb0ba1881202", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + }, + "date": "2014-12-05T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:eb3fe20b-3b1d-43c9-af43-882fc0e8b576", + "resource": { + "resourceType": "Claim", + "id": "eb3fe20b-3b1d-43c9-af43-882fc0e8b576", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "organization": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:9dce2503-5576-49bc-b0df-a1d98ad4bc11" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:664cd96e-79b4-459c-8ba2-85d25bf84108" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a3d039d6-dd93-4d43-a741-6be2ee312375" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:81205a66-b680-4861-b1de-eb0ba1881202" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f16e3769-5904-4842-b69a-a256ada215d1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f16e3769-5904-4842-b69a-a256ada215d1", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "eb3fe20b-3b1d-43c9-af43-882fc0e8b576" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2014-12-05T15:36:32-08:00", + "end": "2015-12-05T15:36:32-08:00" + }, + "created": "2014-12-05T15:36:32-08:00", + "provider": { + "identifier": { + "value": "d08d5908-40a4-3571-8c54-0cc029c8277f" + } + }, + "organization": { + "identifier": { + "value": "c44f361c-2efb-3050-8f97-0354a12e2920" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:eb3fe20b-3b1d-43c9-af43-882fc0e8b576" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-12-05T15:21:32-08:00", + "end": "2014-12-05T15:36:32-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0", + "resource": { + "resourceType": "Encounter", + "id": "dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2015-02-06T15:21:32-08:00", + "end": "2015-02-06T15:51:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ca2090e9-efe2-4c5d-8919-8bdf6012e351", + "resource": { + "resourceType": "Observation", + "id": "ca2090e9-efe2-4c5d-8919-8bdf6012e351", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 73.900, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:633fae99-bec8-4c26-b5d5-a9648508ef07", + "resource": { + "resourceType": "Observation", + "id": "633fae99-bec8-4c26-b5d5-a9648508ef07", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb1527bb-5418-4ec0-9b53-0d74faafd2e2", + "resource": { + "resourceType": "Observation", + "id": "cb1527bb-5418-4ec0-9b53-0d74faafd2e2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 9.2000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3736248b-6e4f-48b6-bd27-e31e42dc3c7e", + "resource": { + "resourceType": "Observation", + "id": "3736248b-6e4f-48b6-bd27-e31e42dc3c7e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 42.908, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:41c607fd-4915-4f99-91c5-74d9995a2a1a", + "resource": { + "resourceType": "Observation", + "id": "41c607fd-4915-4f99-91c5-74d9995a2a1a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 44.570, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:210d1e9b-451c-499a-9f33-39243968b227", + "resource": { + "resourceType": "Observation", + "id": "210d1e9b-451c-499a-9f33-39243968b227", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 117, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c1587e68-1237-41ec-9d4b-4be23b1ef8a3", + "resource": { + "resourceType": "Observation", + "id": "c1587e68-1237-41ec-9d4b-4be23b1ef8a3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 83, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6f26a56-0eec-4790-a9f1-970a363c9ebd", + "resource": { + "resourceType": "Observation", + "id": "c6f26a56-0eec-4790-a9f1-970a363c9ebd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:48a77809-fe5e-4ad9-aab8-aa5975b473a5", + "resource": { + "resourceType": "Observation", + "id": "48a77809-fe5e-4ad9-aab8-aa5975b473a5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "effectiveDateTime": "2015-02-06T15:21:32-08:00", + "issued": "2015-02-06T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1", + "resource": { + "resourceType": "Procedure", + "id": "4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + }, + "performedPeriod": { + "start": "2015-02-06T15:21:32-08:00", + "end": "2015-02-06T15:36:32-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f7fe2351-68e7-41e2-ba8f-b23db05c525c", + "resource": { + "resourceType": "Claim", + "id": "f7fe2351-68e7-41e2-ba8f-b23db05c525c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2015-02-06T15:21:32-08:00", + "end": "2015-02-06T15:51:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 608.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:72c5cafb-5fe0-40e0-a8ea-c6cd4b932ce5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "72c5cafb-5fe0-40e0-a8ea-c6cd4b932ce5", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f7fe2351-68e7-41e2-ba8f-b23db05c525c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-02-06T15:51:32-08:00", + "end": "2016-02-06T15:51:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 121.79200000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 487.16800000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 608.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 608.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 487.16800000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777", + "resource": { + "resourceType": "Encounter", + "id": "9df7d2be-be25-4a9e-a85a-c6955ab53777", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2015-05-08T16:21:32-07:00", + "end": "2015-05-08T16:51:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:29c8089f-303b-486b-984a-4c91269d1f3e", + "resource": { + "resourceType": "Observation", + "id": "29c8089f-303b-486b-984a-4c91269d1f3e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 76.900, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c67927ad-7b8e-4e3f-ae5a-2996c9e73f04", + "resource": { + "resourceType": "Observation", + "id": "c67927ad-7b8e-4e3f-ae5a-2996c9e73f04", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b9a51ae-dff4-40a4-9841-f4360f75d47d", + "resource": { + "resourceType": "Observation", + "id": "4b9a51ae-dff4-40a4-9841-f4360f75d47d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 9.8000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bb8ba01-7438-40f1-b4a3-10888c92a658", + "resource": { + "resourceType": "Observation", + "id": "7bb8ba01-7438-40f1-b4a3-10888c92a658", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 39.248, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ce22474b-e062-4cea-923e-81890efbafe9", + "resource": { + "resourceType": "Observation", + "id": "ce22474b-e062-4cea-923e-81890efbafe9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 45.210, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c008671-044e-42ae-8a20-eb10e727942a", + "resource": { + "resourceType": "Observation", + "id": "1c008671-044e-42ae-8a20-eb10e727942a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a1d4bed3-7f4d-4956-9bc7-b97e26f9d8b1", + "resource": { + "resourceType": "Observation", + "id": "a1d4bed3-7f4d-4956-9bc7-b97e26f9d8b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 80, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b254991b-a739-415f-98fa-67f214277b38", + "resource": { + "resourceType": "Observation", + "id": "b254991b-a739-415f-98fa-67f214277b38", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50b79e2d-80c5-4253-bd1e-e01def48e309", + "resource": { + "resourceType": "Observation", + "id": "50b79e2d-80c5-4253-bd1e-e01def48e309", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "effectiveDateTime": "2015-05-08T16:21:32-07:00", + "issued": "2015-05-08T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe", + "resource": { + "resourceType": "Procedure", + "id": "edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "performedPeriod": { + "start": "2015-05-08T16:21:32-07:00", + "end": "2015-05-08T16:36:32-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:8288b96b-ce1c-48c2-b4e0-d1ef2b27851e", + "resource": { + "resourceType": "Immunization", + "id": "8288b96b-ce1c-48c2-b4e0-d1ef2b27851e", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + }, + "date": "2015-05-08T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:caa7a6f1-01f5-417c-8f51-38190b61c434", + "resource": { + "resourceType": "Claim", + "id": "caa7a6f1-01f5-417c-8f51-38190b61c434", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2015-05-08T16:21:32-07:00", + "end": "2015-05-08T16:51:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8288b96b-ce1c-48c2-b4e0-d1ef2b27851e" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 795.44, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2764a79a-18f8-49ba-863c-6bd2b7b2bda7", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2764a79a-18f8-49ba-863c-6bd2b7b2bda7", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "caa7a6f1-01f5-417c-8f51-38190b61c434" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-05-08T16:51:32-07:00", + "end": "2016-05-08T16:51:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 159.08800000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 636.3520000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 795.44, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 795.44, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 748.7680000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e", + "resource": { + "resourceType": "Encounter", + "id": "02276df4-3e3e-44ff-87ae-4151d6cfdc6e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2015-11-06T15:21:32-08:00", + "end": "2015-11-06T15:36:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2464911c-0a6b-450e-9e79-157b90b3bd1d", + "resource": { + "resourceType": "Observation", + "id": "2464911c-0a6b-450e-9e79-157b90b3bd1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 82.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed19c96e-d1fd-48ea-8fa7-649e7c29919b", + "resource": { + "resourceType": "Observation", + "id": "ed19c96e-d1fd-48ea-8fa7-649e7c29919b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76e59a72-a598-459f-8385-5dd5748b40aa", + "resource": { + "resourceType": "Observation", + "id": "76e59a72-a598-459f-8385-5dd5748b40aa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 10.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6214e182-c516-4c77-97ed-697c9670d15a", + "resource": { + "resourceType": "Observation", + "id": "6214e182-c516-4c77-97ed-697c9670d15a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 34.919, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9cec71c5-ca38-43b0-8a66-90e57aaf74fe", + "resource": { + "resourceType": "Observation", + "id": "9cec71c5-ca38-43b0-8a66-90e57aaf74fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 46.120, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:444065d3-2a85-4d0e-a068-3de5d0b4de9e", + "resource": { + "resourceType": "Observation", + "id": "444065d3-2a85-4d0e-a068-3de5d0b4de9e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2fabf9b3-2a7f-4a55-a521-5996546d5f99", + "resource": { + "resourceType": "Observation", + "id": "2fabf9b3-2a7f-4a55-a521-5996546d5f99", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 79, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ec2ca22f-bb9a-4a1f-96b8-de93d4e10a19", + "resource": { + "resourceType": "Observation", + "id": "ec2ca22f-bb9a-4a1f-96b8-de93d4e10a19", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7af7414f-6f62-455b-88a8-c3982090522f", + "resource": { + "resourceType": "Observation", + "id": "7af7414f-6f62-455b-88a8-c3982090522f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "effectiveDateTime": "2015-11-06T15:21:32-08:00", + "issued": "2015-11-06T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9bc202b-1079-4d36-898e-5731fa54fcfa", + "resource": { + "resourceType": "Immunization", + "id": "c9bc202b-1079-4d36-898e-5731fa54fcfa", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + }, + "date": "2015-11-06T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a55be85b-839f-44af-b1ae-bec185400bba", + "resource": { + "resourceType": "Claim", + "id": "a55be85b-839f-44af-b1ae-bec185400bba", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2015-11-06T15:21:32-08:00", + "end": "2015-11-06T15:36:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c9bc202b-1079-4d36-898e-5731fa54fcfa" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b1dce11f-7e61-4fab-b324-1570b32e0cc9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b1dce11f-7e61-4fab-b324-1570b32e0cc9", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a55be85b-839f-44af-b1ae-bec185400bba" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-11-06T15:36:32-08:00", + "end": "2016-11-06T15:36:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c", + "resource": { + "resourceType": "Encounter", + "id": "bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2016-05-06T16:21:32-07:00", + "end": "2016-05-06T16:51:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4a15ce04-4d5e-40b4-b021-e875a315c38d", + "resource": { + "resourceType": "Observation", + "id": "4a15ce04-4d5e-40b4-b021-e875a315c38d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 86.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e6b135a-62c1-40f0-8b2d-1b5b93ce9544", + "resource": { + "resourceType": "Observation", + "id": "4e6b135a-62c1-40f0-8b2d-1b5b93ce9544", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:efc22b1e-f5eb-41da-9a21-8f807beaa716", + "resource": { + "resourceType": "Observation", + "id": "efc22b1e-f5eb-41da-9a21-8f807beaa716", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 12.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4df3a747-e642-49da-81c8-03e0aff973b5", + "resource": { + "resourceType": "Observation", + "id": "4df3a747-e642-49da-81c8-03e0aff973b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 53.504, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82cb153b-243e-4956-9bc1-1dab9c77aaab", + "resource": { + "resourceType": "Observation", + "id": "82cb153b-243e-4956-9bc1-1dab9c77aaab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 46.730, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:04b39abd-bb56-49da-9b14-acdc84b108d0", + "resource": { + "resourceType": "Observation", + "id": "04b39abd-bb56-49da-9b14-acdc84b108d0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 16.320, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f3c0bda-f6eb-40e5-98ac-696dd4158921", + "resource": { + "resourceType": "Observation", + "id": "1f3c0bda-f6eb-40e5-98ac-696dd4158921", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 57.496, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a63694d8-2566-4b9b-9af1-a416de6af074", + "resource": { + "resourceType": "Observation", + "id": "a63694d8-2566-4b9b-9af1-a416de6af074", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 105, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:015eb094-4fe7-4f44-bc06-80f09399a807", + "resource": { + "resourceType": "Observation", + "id": "015eb094-4fe7-4f44-bc06-80f09399a807", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 64, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db193a6c-a16d-4bf7-a2ae-42000e0d85bb", + "resource": { + "resourceType": "Observation", + "id": "db193a6c-a16d-4bf7-a2ae-42000e0d85bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cca53e11-8465-435e-b17d-8304bc48b70c", + "resource": { + "resourceType": "Observation", + "id": "cca53e11-8465-435e-b17d-8304bc48b70c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "effectiveDateTime": "2016-05-06T16:21:32-07:00", + "issued": "2016-05-06T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:31252f5d-b0ec-4505-b256-2507d4f838e1", + "resource": { + "resourceType": "Procedure", + "id": "31252f5d-b0ec-4505-b256-2507d4f838e1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "performedPeriod": { + "start": "2016-05-06T16:21:32-07:00", + "end": "2016-05-06T16:36:32-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:6480a0d3-7dcd-45f4-ae49-2e274b6b3877", + "resource": { + "resourceType": "Immunization", + "id": "6480a0d3-7dcd-45f4-ae49-2e274b6b3877", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + }, + "date": "2016-05-06T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a8c4cb23-c849-4b5a-a147-cbf51a070d7d", + "resource": { + "resourceType": "Claim", + "id": "a8c4cb23-c849-4b5a-a147-cbf51a070d7d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2016-05-06T16:21:32-07:00", + "end": "2016-05-06T16:51:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:6480a0d3-7dcd-45f4-ae49-2e274b6b3877" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:31252f5d-b0ec-4505-b256-2507d4f838e1" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 533.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:98b3240c-d25c-4191-a01e-003c3d600f00", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "98b3240c-d25c-4191-a01e-003c3d600f00", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a8c4cb23-c849-4b5a-a147-cbf51a070d7d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-05-06T16:51:32-07:00", + "end": "2017-05-06T16:51:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 106.702, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 426.808, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 533.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 533.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 539.224, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1", + "resource": { + "resourceType": "Encounter", + "id": "3748df54-cdc9-4672-b442-491ebec3dbd1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2016-11-04T16:21:32-07:00", + "end": "2016-11-04T16:51:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bcafb02d-bd3e-4bd0-806e-16175c4b1ca1", + "resource": { + "resourceType": "Observation", + "id": "bcafb02d-bd3e-4bd0-806e-16175c4b1ca1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 90.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2aacc861-4f5b-4ef0-bd90-d7674826b8cf", + "resource": { + "resourceType": "Observation", + "id": "2aacc861-4f5b-4ef0-bd90-d7674826b8cf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:049730f9-5c82-4073-b2a7-35c74f960006", + "resource": { + "resourceType": "Observation", + "id": "049730f9-5c82-4073-b2a7-35c74f960006", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 13.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:45ed0f26-69a5-4144-a4c3-017512f7d4ec", + "resource": { + "resourceType": "Observation", + "id": "45ed0f26-69a5-4144-a4c3-017512f7d4ec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 67.727, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e64b70ef-1c97-41cb-a9c5-92362a37b8ca", + "resource": { + "resourceType": "Observation", + "id": "e64b70ef-1c97-41cb-a9c5-92362a37b8ca", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 47.150, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68ca7411-c0ff-4d06-b123-3202a7b9ca12", + "resource": { + "resourceType": "Observation", + "id": "68ca7411-c0ff-4d06-b123-3202a7b9ca12", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 16.520, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9221b296-56c5-4a61-8802-6cacf107e5ac", + "resource": { + "resourceType": "Observation", + "id": "9221b296-56c5-4a61-8802-6cacf107e5ac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 71.965, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b185f5c-01f5-40c1-a1d7-bbf4edb054ac", + "resource": { + "resourceType": "Observation", + "id": "6b185f5c-01f5-40c1-a1d7-bbf4edb054ac", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:695086c1-30a2-4c0f-9ec0-5d03e2280ee6", + "resource": { + "resourceType": "Observation", + "id": "695086c1-30a2-4c0f-9ec0-5d03e2280ee6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3eb1df1b-03c9-41dd-9ab4-92233babedd2", + "resource": { + "resourceType": "Observation", + "id": "3eb1df1b-03c9-41dd-9ab4-92233babedd2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6b133cbf-5aea-4a88-babb-36140361eb73", + "resource": { + "resourceType": "Observation", + "id": "6b133cbf-5aea-4a88-babb-36140361eb73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "effectiveDateTime": "2016-11-04T16:21:32-07:00", + "issued": "2016-11-04T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc05be7e-a85d-4dd9-828f-608cd633dc06", + "resource": { + "resourceType": "Procedure", + "id": "dc05be7e-a85d-4dd9-828f-608cd633dc06", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "performedPeriod": { + "start": "2016-11-04T16:21:32-07:00", + "end": "2016-11-04T16:36:32-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:dd99667c-3321-4113-90c1-7f3a98eb7df8", + "resource": { + "resourceType": "Immunization", + "id": "dd99667c-3321-4113-90c1-7f3a98eb7df8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + }, + "date": "2016-11-04T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7e3ae344-23f3-4e7e-97ae-f475dd8c8d70", + "resource": { + "resourceType": "Claim", + "id": "7e3ae344-23f3-4e7e-97ae-f475dd8c8d70", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2016-11-04T16:21:32-07:00", + "end": "2016-11-04T16:51:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:dd99667c-3321-4113-90c1-7f3a98eb7df8" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:dc05be7e-a85d-4dd9-828f-608cd633dc06" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 590.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:83ec271b-be55-4234-893b-05e07574e11a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "83ec271b-be55-4234-893b-05e07574e11a", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7e3ae344-23f3-4e7e-97ae-f475dd8c8d70" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-11-04T16:51:32-07:00", + "end": "2017-11-04T16:51:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 118.184, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 472.736, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 590.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 590.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 585.152, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53", + "resource": { + "resourceType": "Encounter", + "id": "bcc723ef-94f8-4e94-8d33-cc6ae15f1e53", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2017-05-05T16:21:32-07:00", + "end": "2017-05-05T16:36:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3d7c95a1-205f-4ff1-a65c-58f1f52f5669", + "resource": { + "resourceType": "Observation", + "id": "3d7c95a1-205f-4ff1-a65c-58f1f52f5669", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 93.300, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5bee4bf-65bd-461c-9f2f-13d9423d5146", + "resource": { + "resourceType": "Observation", + "id": "f5bee4bf-65bd-461c-9f2f-13d9423d5146", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8375b59-43b1-4a84-9720-1e5b6fa73ff2", + "resource": { + "resourceType": "Observation", + "id": "b8375b59-43b1-4a84-9720-1e5b6fa73ff2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 14.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7284dfb-e35e-4da9-a54b-cfe4d5ea3b7c", + "resource": { + "resourceType": "Observation", + "id": "b7284dfb-e35e-4da9-a54b-cfe4d5ea3b7c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 66.436, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:806ddaa6-b8f1-4935-bf6c-4108430ef673", + "resource": { + "resourceType": "Observation", + "id": "806ddaa6-b8f1-4935-bf6c-4108430ef673", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 47.180, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:61819254-bd76-4a89-b804-bdc49b7c6baf", + "resource": { + "resourceType": "Observation", + "id": "61819254-bd76-4a89-b804-bdc49b7c6baf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 16.230, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7256b2e-87b9-4b62-8e63-9073fb478327", + "resource": { + "resourceType": "Observation", + "id": "a7256b2e-87b9-4b62-8e63-9073fb478327", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 71.086, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a6db32f1-e01f-4faa-986b-e09986deb3e3", + "resource": { + "resourceType": "Observation", + "id": "a6db32f1-e01f-4faa-986b-e09986deb3e3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 124, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:96c055e8-f34e-4b0e-af40-c2a5f4430534", + "resource": { + "resourceType": "Observation", + "id": "96c055e8-f34e-4b0e-af40-c2a5f4430534", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e9b59076-75e2-4f17-bbad-67adfc435488", + "resource": { + "resourceType": "Observation", + "id": "e9b59076-75e2-4f17-bbad-67adfc435488", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f1a2600c-f975-4da3-86cb-ad17fb7b9d6b", + "resource": { + "resourceType": "Observation", + "id": "f1a2600c-f975-4da3-86cb-ad17fb7b9d6b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + }, + "effectiveDateTime": "2017-05-05T16:21:32-07:00", + "issued": "2017-05-05T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fa5a7301-20e3-450d-b0ce-e0bfdc1bb814", + "resource": { + "resourceType": "Claim", + "id": "fa5a7301-20e3-450d-b0ce-e0bfdc1bb814", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2017-05-05T16:21:32-07:00", + "end": "2017-05-05T16:36:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f204debd-3e15-49cd-a0eb-45274af5d0be", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f204debd-3e15-49cd-a0eb-45274af5d0be", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "fa5a7301-20e3-450d-b0ce-e0bfdc1bb814" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2017-05-05T16:36:32-07:00", + "end": "2018-05-05T16:36:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b", + "resource": { + "resourceType": "Encounter", + "id": "d2763c1d-b19b-47bb-b734-9c49421c839b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2017-11-03T16:21:32-07:00", + "end": "2017-11-03T16:36:32-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:45f5a054-e562-4041-851d-f50a637a57fc", + "resource": { + "resourceType": "Observation", + "id": "45f5a054-e562-4041-851d-f50a637a57fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 96.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0895469c-81e2-41ec-944e-563cc2cd9022", + "resource": { + "resourceType": "Observation", + "id": "0895469c-81e2-41ec-944e-563cc2cd9022", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd9e7976-9c07-43b8-8532-b613c74522f9", + "resource": { + "resourceType": "Observation", + "id": "cd9e7976-9c07-43b8-8532-b613c74522f9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 14.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:293885c5-daa6-49f1-a66b-46b854ccec7f", + "resource": { + "resourceType": "Observation", + "id": "293885c5-daa6-49f1-a66b-46b854ccec7f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 66.436, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30b28a63-4ffa-49b9-a1c5-35764d2bc5b1", + "resource": { + "resourceType": "Observation", + "id": "30b28a63-4ffa-49b9-a1c5-35764d2bc5b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 47.180, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cc527b5-99ca-492c-97df-d1f5b0b373d5", + "resource": { + "resourceType": "Observation", + "id": "0cc527b5-99ca-492c-97df-d1f5b0b373d5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 15.890, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc1b1af5-9d52-4828-aa62-279a5f48a085", + "resource": { + "resourceType": "Observation", + "id": "fc1b1af5-9d52-4828-aa62-279a5f48a085", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 67.013, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8de69a0f-6463-4e45-a347-b8acb729ff60", + "resource": { + "resourceType": "Observation", + "id": "8de69a0f-6463-4e45-a347-b8acb729ff60", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 119, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd2d183c-53ff-427a-9ca9-9ccfbd2bab07", + "resource": { + "resourceType": "Observation", + "id": "dd2d183c-53ff-427a-9ca9-9ccfbd2bab07", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 64, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3fcf856f-ead4-45a4-b338-4dcafb1bb80e", + "resource": { + "resourceType": "Observation", + "id": "3fcf856f-ead4-45a4-b338-4dcafb1bb80e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:985cb719-8660-4129-9eec-f4b4cf9983b0", + "resource": { + "resourceType": "Observation", + "id": "985cb719-8660-4129-9eec-f4b4cf9983b0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "effectiveDateTime": "2017-11-03T16:21:32-07:00", + "issued": "2017-11-03T16:21:32.447-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed6030da-081e-4ff2-9765-ea61f3c734f4", + "resource": { + "resourceType": "Immunization", + "id": "ed6030da-081e-4ff2-9765-ea61f3c734f4", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + }, + "date": "2017-11-03T16:21:32-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:266dfe29-12d3-4744-a862-f2f1c8aa33e0", + "resource": { + "resourceType": "Claim", + "id": "266dfe29-12d3-4744-a862-f2f1c8aa33e0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2017-11-03T16:21:32-07:00", + "end": "2017-11-03T16:36:32-07:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ed6030da-081e-4ff2-9765-ea61f3c734f4" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4d408cf9-09d1-4007-bc07-76cbaefda1c1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4d408cf9-09d1-4007-bc07-76cbaefda1c1", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "266dfe29-12d3-4744-a862-f2f1c8aa33e0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2017-11-03T16:36:32-07:00", + "end": "2018-11-03T16:36:32-07:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b", + "resource": { + "resourceType": "Encounter", + "id": "380edf2f-79ab-481f-92fa-153d501d5e3b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2018-11-09T15:21:32-08:00", + "end": "2018-11-09T15:51:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:96d199a3-5b05-4893-8261-4c20493535dc", + "resource": { + "resourceType": "Observation", + "id": "96d199a3-5b05-4893-8261-4c20493535dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 103.10, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:304e7a9e-9ab0-4df9-8a03-feac0be792fc", + "resource": { + "resourceType": "Observation", + "id": "304e7a9e-9ab0-4df9-8a03-feac0be792fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ffca0d8d-0f87-4004-93ba-553f21e25709", + "resource": { + "resourceType": "Observation", + "id": "ffca0d8d-0f87-4004-93ba-553f21e25709", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 16.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:003ebfe2-cd4a-4c2a-bbd3-732a9eb1343a", + "resource": { + "resourceType": "Observation", + "id": "003ebfe2-cd4a-4c2a-bbd3-732a9eb1343a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 15.100, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f90e75b2-44b6-4497-9a72-58137a1deb39", + "resource": { + "resourceType": "Observation", + "id": "f90e75b2-44b6-4497-9a72-58137a1deb39", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 48.190, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7d08d1fe-e937-43d4-a470-3dccb2eea9b5", + "resource": { + "resourceType": "Observation", + "id": "7d08d1fe-e937-43d4-a470-3dccb2eea9b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2f3393b3-b75d-443f-932e-6f216fe6bcdb", + "resource": { + "resourceType": "Observation", + "id": "2f3393b3-b75d-443f-932e-6f216fe6bcdb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 86, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49e6af52-bf0c-4766-9060-bebc09efd71a", + "resource": { + "resourceType": "Observation", + "id": "49e6af52-bf0c-4766-9060-bebc09efd71a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8315e0fb-ac6c-4b85-99e8-0c09f32b5bd3", + "resource": { + "resourceType": "Observation", + "id": "8315e0fb-ac6c-4b85-99e8-0c09f32b5bd3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "effectiveDateTime": "2018-11-09T15:21:32-08:00", + "issued": "2018-11-09T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c706176f-4627-4e21-bb9d-75785b155ed9", + "resource": { + "resourceType": "Procedure", + "id": "c706176f-4627-4e21-bb9d-75785b155ed9", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "performedPeriod": { + "start": "2018-11-09T15:21:32-08:00", + "end": "2018-11-09T15:36:32-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c75c5718-8596-40ed-b9d4-3e2cad66bcdd", + "resource": { + "resourceType": "Immunization", + "id": "c75c5718-8596-40ed-b9d4-3e2cad66bcdd", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "date": "2018-11-09T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:56761bf9-a4e5-400f-a96f-7426ef48e3c1", + "resource": { + "resourceType": "Immunization", + "id": "56761bf9-a4e5-400f-a96f-7426ef48e3c1", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "date": "2018-11-09T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0066d618-88a8-4eaf-b66d-9600eb6608e6", + "resource": { + "resourceType": "Immunization", + "id": "0066d618-88a8-4eaf-b66d-9600eb6608e6", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "date": "2018-11-09T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b6c3ef13-394b-4131-92dd-df2a28ddb4df", + "resource": { + "resourceType": "Immunization", + "id": "b6c3ef13-394b-4131-92dd-df2a28ddb4df", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "date": "2018-11-09T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4e3b4c39-cb94-4c09-b2db-127a25e80426", + "resource": { + "resourceType": "Immunization", + "id": "4e3b4c39-cb94-4c09-b2db-127a25e80426", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + }, + "date": "2018-11-09T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f5173492-5912-4fd2-aa9b-dcf60d6709dd", + "resource": { + "resourceType": "Claim", + "id": "f5173492-5912-4fd2-aa9b-dcf60d6709dd", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2018-11-09T15:21:32-08:00", + "end": "2018-11-09T15:51:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c75c5718-8596-40ed-b9d4-3e2cad66bcdd" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:56761bf9-a4e5-400f-a96f-7426ef48e3c1" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:0066d618-88a8-4eaf-b66d-9600eb6608e6" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:b6c3ef13-394b-4131-92dd-df2a28ddb4df" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4e3b4c39-cb94-4c09-b2db-127a25e80426" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:c706176f-4627-4e21-bb9d-75785b155ed9" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 338.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:aba58129-01b1-459e-87e9-21e505c1cba7", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "aba58129-01b1-459e-87e9-21e505c1cba7", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f5173492-5912-4fd2-aa9b-dcf60d6709dd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-11-09T15:51:32-08:00", + "end": "2019-11-09T15:51:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 67.68400000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 270.73600000000005, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 338.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 338.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 832.816, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504", + "resource": { + "resourceType": "Encounter", + "id": "5bea4746-76ab-4b50-b09d-3337227f8504", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + } + ], + "period": { + "start": "2019-11-15T15:21:32-08:00", + "end": "2019-11-15T15:36:32-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9b2ade99-39f7-4d8c-a92a-3744c38ca0c9", + "resource": { + "resourceType": "Observation", + "id": "9b2ade99-39f7-4d8c-a92a-3744c38ca0c9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 109.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5146147e-2697-4175-8d81-0218ff1ea6e5", + "resource": { + "resourceType": "Observation", + "id": "5146147e-2697-4175-8d81-0218ff1ea6e5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:267e9107-1ad0-4185-8ceb-74fa5a1f047f", + "resource": { + "resourceType": "Observation", + "id": "267e9107-1ad0-4185-8ceb-74fa5a1f047f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 19.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ccca442-c229-4689-bd35-12420ed9c3aa", + "resource": { + "resourceType": "Observation", + "id": "1ccca442-c229-4689-bd35-12420ed9c3aa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 16.230, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e05d098e-a8f2-4a73-a505-512458a346c4", + "resource": { + "resourceType": "Observation", + "id": "e05d098e-a8f2-4a73-a505-512458a346c4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 73.726, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e3f0218-3535-4659-8bb5-114d70255c19", + "resource": { + "resourceType": "Observation", + "id": "6e3f0218-3535-4659-8bb5-114d70255c19", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eab9a943-9467-4d03-bd7f-513d1f93520d", + "resource": { + "resourceType": "Observation", + "id": "eab9a943-9467-4d03-bd7f-513d1f93520d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 88, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7fe674c8-901c-4de2-bc70-5098bb8b9ca5", + "resource": { + "resourceType": "Observation", + "id": "7fe674c8-901c-4de2-bc70-5098bb8b9ca5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3498e489-4ac0-4496-93a1-b765716a6f11", + "resource": { + "resourceType": "Observation", + "id": "3498e489-4ac0-4496-93a1-b765716a6f11", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 9.7329, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:65318aa0-6b55-46d2-880b-b188d5459af3", + "resource": { + "resourceType": "Observation", + "id": "65318aa0-6b55-46d2-880b-b188d5459af3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 5.3382, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", + "resource": { + "resourceType": "Observation", + "id": "bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 16.992, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", + "resource": { + "resourceType": "Observation", + "id": "4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 37.183, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b687de9-a614-481c-ba0d-f1fcf298045d", + "resource": { + "resourceType": "Observation", + "id": "0b687de9-a614-481c-ba0d-f1fcf298045d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 82.592, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b52e4ee4-a6e1-466e-a053-544f494b359c", + "resource": { + "resourceType": "Observation", + "id": "b52e4ee4-a6e1-466e-a053-544f494b359c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 30.831, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:79883a19-1cf1-4e83-ac1a-0986a6e4aed4", + "resource": { + "resourceType": "Observation", + "id": "79883a19-1cf1-4e83-ac1a-0986a6e4aed4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 35.670, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e821119-0284-4061-9305-42a9fbc53df8", + "resource": { + "resourceType": "Observation", + "id": "0e821119-0284-4061-9305-42a9fbc53df8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 42.759, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f528b8c2-be75-4361-8f81-9f2e92c5b337", + "resource": { + "resourceType": "Observation", + "id": "f528b8c2-be75-4361-8f81-9f2e92c5b337", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 428.98, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e021bd53-bf27-439c-a94a-ba9272004957", + "resource": { + "resourceType": "Observation", + "id": "e021bd53-bf27-439c-a94a-ba9272004957", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 280.42, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1295d24b-1ed4-46f1-9f2e-b138413b7348", + "resource": { + "resourceType": "Observation", + "id": "1295d24b-1ed4-46f1-9f2e-b138413b7348", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueQuantity": { + "value": 9.4251, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd782321-ba34-4452-a575-782f2448b7b8", + "resource": { + "resourceType": "Observation", + "id": "dd782321-ba34-4452-a575-782f2448b7b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7425a267-f35e-4ab8-a6bb-38c16ca67268", + "resource": { + "resourceType": "Immunization", + "id": "7425a267-f35e-4ab8-a6bb-38c16ca67268", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "encounter": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "date": "2019-11-15T15:21:32-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8eae1649-e850-4c94-8ed1-bbb619dcafe8", + "resource": { + "resourceType": "DiagnosticReport", + "id": "8eae1649-e850-4c94-8ed1-bbb619dcafe8", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "context": { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + }, + "effectiveDateTime": "2019-11-15T15:21:32-08:00", + "issued": "2019-11-15T15:21:32.447-08:00", + "result": [ + { + "reference": "urn:uuid:3498e489-4ac0-4496-93a1-b765716a6f11", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:65318aa0-6b55-46d2-880b-b188d5459af3", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:0b687de9-a614-481c-ba0d-f1fcf298045d", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:b52e4ee4-a6e1-466e-a053-544f494b359c", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:79883a19-1cf1-4e83-ac1a-0986a6e4aed4", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:0e821119-0284-4061-9305-42a9fbc53df8", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:f528b8c2-be75-4361-8f81-9f2e92c5b337", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:e021bd53-bf27-439c-a94a-ba9272004957", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1295d24b-1ed4-46f1-9f2e-b138413b7348", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:dd5b29e0-824a-4272-8d8b-9649c75edb0c", + "resource": { + "resourceType": "Claim", + "id": "dd5b29e0-824a-4272-8d8b-9649c75edb0c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "start": "2019-11-15T15:21:32-08:00", + "end": "2019-11-15T15:36:32-08:00" + }, + "organization": { + "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7425a267-f35e-4ab8-a6bb-38c16ca67268" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ecba9532-715b-4d57-816a-3f0e00f9a67c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ecba9532-715b-4d57-816a-3f0e00f9a67c", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Anthem" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "dd5b29e0-824a-4272-8d8b-9649c75edb0c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-11-15T15:36:32-08:00", + "end": "2020-11-15T15:36:32-08:00" + }, + "provider": { + "identifier": { + "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" + } + }, + "organization": { + "identifier": { + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json new file mode 100644 index 000000000000..65e224b7d394 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json @@ -0,0 +1,28126 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612", + "resource": { + "resourceType": "Patient", + "id": "136f997d-0a94-4573-97da-a53b5060a612", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -3753024713238705289 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2028-9", + "display": "Asian" + } + }, + { + "url": "text", + "valueString": "Asian" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Fredericka800 Bayer639" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "M" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Hatfield", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.002142737663232659 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 6.997857262336767 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "136f997d-0a94-4573-97da-a53b5060a612" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "136f997d-0a94-4573-97da-a53b5060a612" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-87-6648" + } + ], + "name": [ + { + "use": "official", + "family": "Will178", + "given": [ + "Damon455" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-923-1711", + "use": "home" + } + ], + "gender": "male", + "birthDate": "2012-01-04", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.360699590968395 + }, + { + "url": "longitude", + "valueDecimal": -71.15833176606733 + } + ] + } + ], + "line": [ + "280 Lesch Junction" + ], + "city": "Cambridge", + "state": "Massachusetts", + "postalCode": "02140", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245", + "resource": { + "resourceType": "Organization", + "id": "9bd6ea26-d344-3409-aea7-2e925fd0d245", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP32892", + "telecom": [ + { + "system": "phone", + "value": "617-744-6527" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.369451 + }, + { + "url": "longitude", + "valueDecimal": -71.177925 + } + ] + } + ], + "line": [ + "158 A BELMONT ST" + ], + "city": "WATERTOWN", + "state": "MA", + "postalCode": "02472-2912", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0", + "resource": { + "resourceType": "Practitioner", + "id": "ba314750-701e-3370-ade0-270e178d80e0", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "22240" + } + ], + "active": true, + "name": [ + { + "family": "Pacocha935", + "given": [ + "Ula130" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "158 A BELMONT ST" + ], + "city": "WATERTOWN", + "state": "MA", + "postalCode": "02472-2912", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350", + "resource": { + "resourceType": "Encounter", + "id": "aa114ab8-742d-4737-88a5-1e216870f350", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-01-04T20:05:11-08:00", + "end": "2012-01-04T20:35:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b5264b2e-b210-4b4f-954b-96ab336f1e85", + "resource": { + "resourceType": "Observation", + "id": "b5264b2e-b210-4b4f-954b-96ab336f1e85", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 50.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:829ed4fa-5bec-4d81-acf6-053d0970e678", + "resource": { + "resourceType": "Observation", + "id": "829ed4fa-5bec-4d81-acf6-053d0970e678", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7adaafd6-d242-4d5c-8dcc-45cdc47773f2", + "resource": { + "resourceType": "Observation", + "id": "7adaafd6-d242-4d5c-8dcc-45cdc47773f2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 3.6000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:26e5ad01-5e59-48a0-8e09-7bbe2561f586", + "resource": { + "resourceType": "Observation", + "id": "26e5ad01-5e59-48a0-8e09-7bbe2561f586", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 37.964, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:324535ed-c3cd-4341-baf0-f55c9d76f5e4", + "resource": { + "resourceType": "Observation", + "id": "324535ed-c3cd-4341-baf0-f55c9d76f5e4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 34.180, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:faf8f54d-ecdb-44be-9a56-12d8462f680d", + "resource": { + "resourceType": "Observation", + "id": "faf8f54d-ecdb-44be-9a56-12d8462f680d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:30a7cdef-02ed-46d0-93fa-adacabc8ef98", + "resource": { + "resourceType": "Observation", + "id": "30a7cdef-02ed-46d0-93fa-adacabc8ef98", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 84, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d7e9211-6989-437a-9573-5a631cc52c7a", + "resource": { + "resourceType": "Observation", + "id": "6d7e9211-6989-437a-9573-5a631cc52c7a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", + "resource": { + "resourceType": "Observation", + "id": "ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 6.5440, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3953521d-dced-4d33-9bd7-60c1b3147e6c", + "resource": { + "resourceType": "Observation", + "id": "3953521d-dced-4d33-9bd7-60c1b3147e6c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 5.1374, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f77c37e4-57de-4027-a54d-aa02092f5011", + "resource": { + "resourceType": "Observation", + "id": "f77c37e4-57de-4027-a54d-aa02092f5011", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 16.950, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5797b11d-116a-4a10-92a8-54a137971870", + "resource": { + "resourceType": "Observation", + "id": "5797b11d-116a-4a10-92a8-54a137971870", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 39.246, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:658ad303-82c7-455f-b365-5acb63a8ca74", + "resource": { + "resourceType": "Observation", + "id": "658ad303-82c7-455f-b365-5acb63a8ca74", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 88.304, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:da416a2c-e670-4a06-8197-db911e5213ab", + "resource": { + "resourceType": "Observation", + "id": "da416a2c-e670-4a06-8197-db911e5213ab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 27.571, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", + "resource": { + "resourceType": "Observation", + "id": "2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 33.153, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:365e9ab9-da06-4c1b-a628-7fdb7279e08d", + "resource": { + "resourceType": "Observation", + "id": "365e9ab9-da06-4c1b-a628-7fdb7279e08d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 44.607, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", + "resource": { + "resourceType": "Observation", + "id": "23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 238.70, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aa2c3dad-a1cc-4319-882e-4f270e9448f5", + "resource": { + "resourceType": "Observation", + "id": "aa2c3dad-a1cc-4319-882e-4f270e9448f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 267.69, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29fd3111-9833-4d16-b742-9a88d528958e", + "resource": { + "resourceType": "Observation", + "id": "29fd3111-9833-4d16-b742-9a88d528958e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 11.169, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4d5bd8fc-7cb9-4ca8-b2bb-dcb9a2bf8b69", + "resource": { + "resourceType": "Observation", + "id": "4d5bd8fc-7cb9-4ca8-b2bb-dcb9a2bf8b69", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50116840-3db7-432d-9b08-cd549b005bbf", + "resource": { + "resourceType": "Procedure", + "id": "50116840-3db7-432d-9b08-cd549b005bbf", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "performedPeriod": { + "start": "2012-01-04T20:05:11-08:00", + "end": "2012-01-04T20:20:11-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:947f8511-77e9-4208-b330-56896f4ca20a", + "resource": { + "resourceType": "Immunization", + "id": "947f8511-77e9-4208-b330-56896f4ca20a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "date": "2012-01-04T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:6673c56f-3907-419c-bf9a-cc07efb1e250", + "resource": { + "resourceType": "DiagnosticReport", + "id": "6673c56f-3907-419c-bf9a-cc07efb1e250", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + }, + "effectiveDateTime": "2012-01-04T20:05:11-08:00", + "issued": "2012-01-04T20:05:11.927-08:00", + "result": [ + { + "reference": "urn:uuid:ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:3953521d-dced-4d33-9bd7-60c1b3147e6c", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:f77c37e4-57de-4027-a54d-aa02092f5011", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:5797b11d-116a-4a10-92a8-54a137971870", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:658ad303-82c7-455f-b365-5acb63a8ca74", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:da416a2c-e670-4a06-8197-db911e5213ab", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:365e9ab9-da06-4c1b-a628-7fdb7279e08d", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:aa2c3dad-a1cc-4319-882e-4f270e9448f5", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:29fd3111-9833-4d16-b742-9a88d528958e", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:5e317710-ebf3-4e4d-839b-91dd57dabd65", + "resource": { + "resourceType": "Claim", + "id": "5e317710-ebf3-4e4d-839b-91dd57dabd65", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-01-04T20:05:11-08:00", + "end": "2012-01-04T20:35:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:947f8511-77e9-4208-b330-56896f4ca20a" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:50116840-3db7-432d-9b08-cd549b005bbf" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 865.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:be632d75-e4a3-43ed-ac34-00dad401932e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "be632d75-e4a3-43ed-ac34-00dad401932e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5e317710-ebf3-4e4d-839b-91dd57dabd65" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-01-04T20:35:11-08:00", + "end": "2013-01-04T20:35:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 173.074, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 692.296, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 865.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 865.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 804.7120000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462", + "resource": { + "resourceType": "Encounter", + "id": "61e62625-d724-4f99-9172-5447a5868462", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-02-08T20:05:11-08:00", + "end": "2012-02-08T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:aa656ec3-26ce-4121-a8ef-11e14743b047", + "resource": { + "resourceType": "Observation", + "id": "aa656ec3-26ce-4121-a8ef-11e14743b047", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 54.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:71b88e97-35df-4a1f-bb76-326de8e5c030", + "resource": { + "resourceType": "Observation", + "id": "71b88e97-35df-4a1f-bb76-326de8e5c030", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1332afae-b398-4aa9-8c43-8d0e3de12470", + "resource": { + "resourceType": "Observation", + "id": "1332afae-b398-4aa9-8c43-8d0e3de12470", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 4.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d0772d3-b016-455f-aae8-61e18db764eb", + "resource": { + "resourceType": "Observation", + "id": "6d0772d3-b016-455f-aae8-61e18db764eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 21.966, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eeb7ad30-699c-4a2b-b402-a354bbf48ad2", + "resource": { + "resourceType": "Observation", + "id": "eeb7ad30-699c-4a2b-b402-a354bbf48ad2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 37.850, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:990ad540-ad95-4d8d-aac6-4a7fd2df2811", + "resource": { + "resourceType": "Observation", + "id": "990ad540-ad95-4d8d-aac6-4a7fd2df2811", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2303615-c9da-4937-81fe-a9403e06ea01", + "resource": { + "resourceType": "Observation", + "id": "b2303615-c9da-4937-81fe-a9403e06ea01", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 63, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed840bf4-fa3a-4570-8cf4-9d58bbfb99c2", + "resource": { + "resourceType": "Observation", + "id": "ed840bf4-fa3a-4570-8cf4-9d58bbfb99c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0f3a9ab-5068-4646-9c8c-2d08477b529c", + "resource": { + "resourceType": "Observation", + "id": "b0f3a9ab-5068-4646-9c8c-2d08477b529c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "effectiveDateTime": "2012-02-08T20:05:11-08:00", + "issued": "2012-02-08T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d842fbbd-0a13-4d65-a515-5d85b0831959", + "resource": { + "resourceType": "Immunization", + "id": "d842fbbd-0a13-4d65-a515-5d85b0831959", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + }, + "date": "2012-02-08T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:6163cc16-6018-461f-9cc7-63c4dc2c75f5", + "resource": { + "resourceType": "Claim", + "id": "6163cc16-6018-461f-9cc7-63c4dc2c75f5", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-02-08T20:05:11-08:00", + "end": "2012-02-08T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:d842fbbd-0a13-4d65-a515-5d85b0831959" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4bf41bb7-51dd-4093-b96c-caad9726af3d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4bf41bb7-51dd-4093-b96c-caad9726af3d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6163cc16-6018-461f-9cc7-63c4dc2c75f5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-02-08T20:20:11-08:00", + "end": "2013-02-08T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766", + "resource": { + "resourceType": "Encounter", + "id": "64cbe9ca-4dc1-4fad-9ec3-d20955aef766", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-04-11T21:05:11-07:00", + "end": "2012-04-11T21:20:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:86bea6a6-416d-45e9-99c3-24cf7df16084", + "resource": { + "resourceType": "Observation", + "id": "86bea6a6-416d-45e9-99c3-24cf7df16084", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 60.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb9238b1-f746-4d46-b014-fc6e46a4f633", + "resource": { + "resourceType": "Observation", + "id": "cb9238b1-f746-4d46-b014-fc6e46a4f633", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:15c9a3c1-c246-46c2-959c-ce7f332d9c69", + "resource": { + "resourceType": "Observation", + "id": "15c9a3c1-c246-46c2-959c-ce7f332d9c69", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 5.9000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:240b9973-55f3-4cea-a66c-96b4a08e1eea", + "resource": { + "resourceType": "Observation", + "id": "240b9973-55f3-4cea-a66c-96b4a08e1eea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 40.884, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:debaa3ea-31cc-4cf1-93a7-314add05ad2a", + "resource": { + "resourceType": "Observation", + "id": "debaa3ea-31cc-4cf1-93a7-314add05ad2a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 40.580, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9a1904d6-d4c0-4077-9f58-ec7dcf74cc1d", + "resource": { + "resourceType": "Observation", + "id": "9a1904d6-d4c0-4077-9f58-ec7dcf74cc1d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 125, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:023bd975-eb45-4b07-9ee8-81ff5f32f708", + "resource": { + "resourceType": "Observation", + "id": "023bd975-eb45-4b07-9ee8-81ff5f32f708", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 70, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d0874b36-2a04-427c-ad73-ebe6f69bbd0a", + "resource": { + "resourceType": "Observation", + "id": "d0874b36-2a04-427c-ad73-ebe6f69bbd0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3459c9f8-4eed-44ae-89e0-62ea0770bc72", + "resource": { + "resourceType": "Observation", + "id": "3459c9f8-4eed-44ae-89e0-62ea0770bc72", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "effectiveDateTime": "2012-04-11T21:05:11-07:00", + "issued": "2012-04-11T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f060c57b-71d2-40b0-8f5d-3a2fafb6fd94", + "resource": { + "resourceType": "Immunization", + "id": "f060c57b-71d2-40b0-8f5d-3a2fafb6fd94", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "date": "2012-04-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:51b7dff3-cdb3-4041-a1b4-11ad92001833", + "resource": { + "resourceType": "Immunization", + "id": "51b7dff3-cdb3-4041-a1b4-11ad92001833", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "date": "2012-04-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c09e6b38-a2ac-46f5-9575-b65025314de7", + "resource": { + "resourceType": "Immunization", + "id": "c09e6b38-a2ac-46f5-9575-b65025314de7", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "date": "2012-04-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:14ddc70f-6c5e-4c36-a1f9-56f673275141", + "resource": { + "resourceType": "Immunization", + "id": "14ddc70f-6c5e-4c36-a1f9-56f673275141", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "date": "2012-04-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:3260d808-6e20-4e34-bca4-e34d08f54147", + "resource": { + "resourceType": "Immunization", + "id": "3260d808-6e20-4e34-bca4-e34d08f54147", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + }, + "date": "2012-04-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0431adb2-45f2-4c54-a6f9-59fb0d69cda6", + "resource": { + "resourceType": "Claim", + "id": "0431adb2-45f2-4c54-a6f9-59fb0d69cda6", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-04-11T21:05:11-07:00", + "end": "2012-04-11T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f060c57b-71d2-40b0-8f5d-3a2fafb6fd94" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:51b7dff3-cdb3-4041-a1b4-11ad92001833" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c09e6b38-a2ac-46f5-9575-b65025314de7" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:14ddc70f-6c5e-4c36-a1f9-56f673275141" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:3260d808-6e20-4e34-bca4-e34d08f54147" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fb411ea0-7750-45af-9df4-51e68aea0a60", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "fb411ea0-7750-45af-9df4-51e68aea0a60", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0431adb2-45f2-4c54-a6f9-59fb0d69cda6" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-04-11T21:20:11-07:00", + "end": "2013-04-11T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890", + "resource": { + "resourceType": "Encounter", + "id": "2941c94b-b52e-4699-943c-1674670cf890", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-06-13T21:05:11-07:00", + "end": "2012-06-13T21:35:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a0c4b166-927a-4e09-a052-abe681ce9181", + "resource": { + "resourceType": "Observation", + "id": "a0c4b166-927a-4e09-a052-abe681ce9181", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 64.100, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c2668ad1-d11f-41c7-8afc-da4535cd4f4b", + "resource": { + "resourceType": "Observation", + "id": "c2668ad1-d11f-41c7-8afc-da4535cd4f4b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:49e55983-39ec-40b3-8855-4830f2edbf5a", + "resource": { + "resourceType": "Observation", + "id": "49e55983-39ec-40b3-8855-4830f2edbf5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 7.1000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e393ff85-49a0-4dd5-a9b8-af2cc7d9b11b", + "resource": { + "resourceType": "Observation", + "id": "e393ff85-49a0-4dd5-a9b8-af2cc7d9b11b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 53.430, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a100883-6942-4c35-a051-0a5e86babba0", + "resource": { + "resourceType": "Observation", + "id": "1a100883-6942-4c35-a051-0a5e86babba0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 42.310, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4302df08-3c6f-4b2b-9ec7-ef584b2c62ab", + "resource": { + "resourceType": "Observation", + "id": "4302df08-3c6f-4b2b-9ec7-ef584b2c62ab", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 132, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c5b5c5ee-3a41-45c4-b5dc-13c6606b5185", + "resource": { + "resourceType": "Observation", + "id": "c5b5c5ee-3a41-45c4-b5dc-13c6606b5185", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e181255b-23ff-49e5-b04a-1a2a7a8b88a3", + "resource": { + "resourceType": "Observation", + "id": "e181255b-23ff-49e5-b04a-1a2a7a8b88a3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f2f7cfe-1156-4616-88a6-38aa826f2857", + "resource": { + "resourceType": "Observation", + "id": "9f2f7cfe-1156-4616-88a6-38aa826f2857", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "effectiveDateTime": "2012-06-13T21:05:11-07:00", + "issued": "2012-06-13T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d38dd1e-5695-487c-b565-e685839e7149", + "resource": { + "resourceType": "Procedure", + "id": "6d38dd1e-5695-487c-b565-e685839e7149", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "performedPeriod": { + "start": "2012-06-13T21:05:11-07:00", + "end": "2012-06-13T21:20:11-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:44758e37-5794-4ca8-b022-1f11b0cbde52", + "resource": { + "resourceType": "Immunization", + "id": "44758e37-5794-4ca8-b022-1f11b0cbde52", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "date": "2012-06-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:21a76df9-25e4-442a-ad18-81a22f061db8", + "resource": { + "resourceType": "Immunization", + "id": "21a76df9-25e4-442a-ad18-81a22f061db8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "119", + "display": "rotavirus, monovalent" + } + ], + "text": "rotavirus, monovalent" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "date": "2012-06-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4550bd48-907d-4b0c-95a0-a18bffdb0251", + "resource": { + "resourceType": "Immunization", + "id": "4550bd48-907d-4b0c-95a0-a18bffdb0251", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "date": "2012-06-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ab501ee9-b4ac-4264-833e-03fad494bc15", + "resource": { + "resourceType": "Immunization", + "id": "ab501ee9-b4ac-4264-833e-03fad494bc15", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "date": "2012-06-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:cff4121b-3f50-4689-be60-3aeacfc9e0c1", + "resource": { + "resourceType": "Immunization", + "id": "cff4121b-3f50-4689-be60-3aeacfc9e0c1", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + }, + "date": "2012-06-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0baab2f5-8d37-48a1-8819-239b9ee5a693", + "resource": { + "resourceType": "Claim", + "id": "0baab2f5-8d37-48a1-8819-239b9ee5a693", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-06-13T21:05:11-07:00", + "end": "2012-06-13T21:35:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:44758e37-5794-4ca8-b022-1f11b0cbde52" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:21a76df9-25e4-442a-ad18-81a22f061db8" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4550bd48-907d-4b0c-95a0-a18bffdb0251" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:ab501ee9-b4ac-4264-833e-03fad494bc15" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:cff4121b-3f50-4689-be60-3aeacfc9e0c1" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:6d38dd1e-5695-487c-b565-e685839e7149" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 7, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 715.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4d8955ab-386a-4c9a-bffa-6ef71758d8f5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4d8955ab-386a-4c9a-bffa-6ef71758d8f5", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0baab2f5-8d37-48a1-8819-239b9ee5a693" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-06-13T21:35:11-07:00", + "end": "2013-06-13T21:35:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 7, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 143.084, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 572.336, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 715.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 715.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1134.4160000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c", + "resource": { + "resourceType": "Encounter", + "id": "ee5c3a54-49f2-4e2b-910c-648693b9262c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-09-12T21:05:11-07:00", + "end": "2012-09-12T21:20:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4958970a-112e-4948-835d-153f4ced6c47", + "resource": { + "resourceType": "Observation", + "id": "4958970a-112e-4948-835d-153f4ced6c47", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 68.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8d61710f-0619-4525-aa01-53ac24a357d0", + "resource": { + "resourceType": "Observation", + "id": "8d61710f-0619-4525-aa01-53ac24a357d0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a09ec90-77cb-4592-b8c2-18728637f6c7", + "resource": { + "resourceType": "Observation", + "id": "5a09ec90-77cb-4592-b8c2-18728637f6c7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 8.4000, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66d4de90-3817-4805-b2ca-4c8a3c003982", + "resource": { + "resourceType": "Observation", + "id": "66d4de90-3817-4805-b2ca-4c8a3c003982", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 55.623, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e40d929e-2055-422b-9e7b-01298d802388", + "resource": { + "resourceType": "Observation", + "id": "e40d929e-2055-422b-9e7b-01298d802388", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 44.020, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ac68d0c5-92c1-44b9-a234-97d86210e7f7", + "resource": { + "resourceType": "Observation", + "id": "ac68d0c5-92c1-44b9-a234-97d86210e7f7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 130, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c26c785-58df-46a2-bd4b-84ee1c08c5fc", + "resource": { + "resourceType": "Observation", + "id": "7c26c785-58df-46a2-bd4b-84ee1c08c5fc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c48bc12c-275c-437f-a814-ac215a465332", + "resource": { + "resourceType": "Observation", + "id": "c48bc12c-275c-437f-a814-ac215a465332", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7582e76f-3d45-4e06-8cfc-5b02ad752905", + "resource": { + "resourceType": "Observation", + "id": "7582e76f-3d45-4e06-8cfc-5b02ad752905", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "effectiveDateTime": "2012-09-12T21:05:11-07:00", + "issued": "2012-09-12T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a4460d65-8bf3-4a16-84ca-824a41f83bdc", + "resource": { + "resourceType": "Immunization", + "id": "a4460d65-8bf3-4a16-84ca-824a41f83bdc", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "date": "2012-09-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc", + "resource": { + "resourceType": "Immunization", + "id": "71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "date": "2012-09-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7385d405-755e-42e8-bb5b-6ff9796849c8", + "resource": { + "resourceType": "Immunization", + "id": "7385d405-755e-42e8-bb5b-6ff9796849c8", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "date": "2012-09-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:3a9b8b01-3702-4f7e-b547-7ad4c4431e09", + "resource": { + "resourceType": "Immunization", + "id": "3a9b8b01-3702-4f7e-b547-7ad4c4431e09", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "date": "2012-09-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:784285d9-3d56-4283-a96c-a1ce50c9795d", + "resource": { + "resourceType": "Immunization", + "id": "784285d9-3d56-4283-a96c-a1ce50c9795d", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "08", + "display": "Hep B, adolescent or pediatric" + } + ], + "text": "Hep B, adolescent or pediatric" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + }, + "date": "2012-09-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e2a64179-4ad9-47ef-92c2-f46d200f41cd", + "resource": { + "resourceType": "Claim", + "id": "e2a64179-4ad9-47ef-92c2-f46d200f41cd", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-09-12T21:05:11-07:00", + "end": "2012-09-12T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a4460d65-8bf3-4a16-84ca-824a41f83bdc" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7385d405-755e-42e8-bb5b-6ff9796849c8" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:3a9b8b01-3702-4f7e-b547-7ad4c4431e09" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:784285d9-3d56-4283-a96c-a1ce50c9795d" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:61eabf69-4783-4497-8104-0838dcccbb1e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "61eabf69-4783-4497-8104-0838dcccbb1e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e2a64179-4ad9-47ef-92c2-f46d200f41cd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-09-12T21:20:11-07:00", + "end": "2013-09-12T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd", + "resource": { + "resourceType": "Encounter", + "id": "63b8bb02-ddc1-4dd6-9aba-8143862a16dd", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2012-12-12T20:05:11-08:00", + "end": "2012-12-12T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:13499c66-624f-430a-9dd6-e3dbe3559fb9", + "resource": { + "resourceType": "Observation", + "id": "13499c66-624f-430a-9dd6-e3dbe3559fb9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 72.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4072583b-147b-49dd-99b3-79b677098b91", + "resource": { + "resourceType": "Observation", + "id": "4072583b-147b-49dd-99b3-79b677098b91", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dd90a3df-9750-4d8b-893f-97675b77e789", + "resource": { + "resourceType": "Observation", + "id": "dd90a3df-9750-4d8b-893f-97675b77e789", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 9.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2a2ad20-4ae5-4323-af38-3ee3acc17913", + "resource": { + "resourceType": "Observation", + "id": "b2a2ad20-4ae5-4323-af38-3ee3acc17913", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 62.032, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51ecf38e-63ff-4682-bebe-94dca4ac816f", + "resource": { + "resourceType": "Observation", + "id": "51ecf38e-63ff-4682-bebe-94dca4ac816f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 45.170, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5172691d-9141-4fa4-a095-993cc74512ad", + "resource": { + "resourceType": "Observation", + "id": "5172691d-9141-4fa4-a095-993cc74512ad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bac3973f-718e-4c4f-a0c6-ed166de0eeec", + "resource": { + "resourceType": "Observation", + "id": "bac3973f-718e-4c4f-a0c6-ed166de0eeec", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 68, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29f854bc-0dc7-4f46-a715-ee20acf18402", + "resource": { + "resourceType": "Observation", + "id": "29f854bc-0dc7-4f46-a715-ee20acf18402", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:257317c7-8d18-4847-aba6-04d096e5d9bc", + "resource": { + "resourceType": "Observation", + "id": "257317c7-8d18-4847-aba6-04d096e5d9bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + }, + "effectiveDateTime": "2012-12-12T20:05:11-08:00", + "issued": "2012-12-12T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b7ca4d8-7ebe-4560-b437-fd5eb339b970", + "resource": { + "resourceType": "Claim", + "id": "0b7ca4d8-7ebe-4560-b437-fd5eb339b970", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-12-12T20:05:11-08:00", + "end": "2012-12-12T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7772f870-a93c-414b-b31e-c57e535a175f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7772f870-a93c-414b-b31e-c57e535a175f", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0b7ca4d8-7ebe-4560-b437-fd5eb339b970" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-12-12T20:20:11-08:00", + "end": "2013-12-12T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", + "resource": { + "resourceType": "Practitioner", + "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10" + } + ], + "active": true, + "name": [ + { + "family": "Jenkins714", + "given": [ + "Lara964" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8", + "resource": { + "resourceType": "Encounter", + "id": "75ca5635-6ff9-42a4-a46c-680aeaa6f8b8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7", + "resource": { + "resourceType": "Condition", + "id": "9e07b9b7-8c28-412f-91d3-b29bde62f6d7", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + }, + "onsetDateTime": "2012-12-29T20:05:11-08:00", + "abatementDateTime": "2013-03-13T21:05:11-07:00", + "assertedDate": "2012-12-29T20:05:11-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:7708603b-7957-4c1e-bfe7-7dd7b25b9290", + "resource": { + "resourceType": "MedicationRequest", + "id": "7708603b-7957-4c1e-bfe7-7dd7b25b9290", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308182", + "display": "Amoxicillin 250 MG Oral Capsule" + } + ], + "text": "Amoxicillin 250 MG Oral Capsule" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + }, + "authoredOn": "2012-12-29T20:05:11-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ec0bf3ae-9dec-420b-9610-229303b44277", + "resource": { + "resourceType": "Claim", + "id": "ec0bf3ae-9dec-420b-9610-229303b44277", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:7708603b-7957-4c1e-bfe7-7dd7b25b9290" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + } + ] + } + ], + "total": { + "value": 3.04, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:938094f3-13b0-451d-b96b-8444272f977b", + "resource": { + "resourceType": "MedicationRequest", + "id": "938094f3-13b0-451d-b96b-8444272f977b", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "198405", + "display": "Ibuprofen 100 MG Oral Tablet" + } + ], + "text": "Ibuprofen 100 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + }, + "authoredOn": "2012-12-29T20:05:11-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:f99eb403-ae1e-4390-8e51-5d4402143221", + "resource": { + "resourceType": "Claim", + "id": "f99eb403-ae1e-4390-8e51-5d4402143221", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:938094f3-13b0-451d-b96b-8444272f977b" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + } + ] + } + ], + "total": { + "value": 40.83, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9", + "resource": { + "resourceType": "Claim", + "id": "1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e4644fb1-bfb4-4ce2-8dbb-561067ea2116", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e4644fb1-bfb4-4ce2-8dbb-561067ea2116", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2012-12-29T20:20:11-08:00", + "end": "2013-12-29T20:20:11-08:00" + }, + "created": "2012-12-29T20:20:11-08:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2012-12-29T20:05:11-08:00", + "end": "2012-12-29T20:20:11-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "resource": { + "resourceType": "Organization", + "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAMBRIDGE HEALTH ALLIANCE", + "telecom": [ + { + "system": "phone", + "value": "6176652300" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", + "resource": { + "resourceType": "Practitioner", + "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "40" + } + ], + "active": true, + "name": [ + { + "family": "Gibson10", + "given": [ + "Loretta235" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced", + "resource": { + "resourceType": "Encounter", + "id": "168f43cb-c0ab-4c7d-bd81-5283c6f52ced", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2013-01-31T20:05:11-08:00", + "end": "2013-01-31T20:35:11-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", + "resource": { + "resourceType": "Condition", + "id": "49b1a980-3b3c-48c5-86b8-7c8043daccd9", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" + }, + "onsetDateTime": "2013-01-31T20:05:11-08:00", + "abatementDateTime": "2013-02-11T20:05:11-08:00", + "assertedDate": "2013-01-31T20:05:11-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:f7dd6c01-fa5b-4285-aa86-ae241f4f695f", + "resource": { + "resourceType": "Observation", + "id": "f7dd6c01-fa5b-4285-aa86-ae241f4f695f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" + }, + "effectiveDateTime": "2013-01-31T20:05:11-08:00", + "issued": "2013-01-31T20:05:11.927-08:00", + "valueQuantity": { + "value": 37.100, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9dbe7b30-e60a-4fb3-bed0-751e50b990ae", + "resource": { + "resourceType": "Procedure", + "id": "9dbe7b30-e60a-4fb3-bed0-751e50b990ae", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" + }, + "performedPeriod": { + "start": "2013-01-31T20:05:11-08:00", + "end": "2013-01-31T20:20:11-08:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", + "display": "Acute viral pharyngitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:82c4ce0c-983b-4c32-b6d6-cba1e44ff824", + "resource": { + "resourceType": "Claim", + "id": "82c4ce0c-983b-4c32-b6d6-cba1e44ff824", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-01-31T20:05:11-08:00", + "end": "2013-01-31T20:35:11-08:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:9dbe7b30-e60a-4fb3-bed0-751e50b990ae" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "net": { + "value": 3121.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f1f8ee6c-acac-40fa-822f-f532927e5a3f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f1f8ee6c-acac-40fa-822f-f532927e5a3f", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "82c4ce0c-983b-4c32-b6d6-cba1e44ff824" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-01-31T20:35:11-08:00", + "end": "2014-01-31T20:35:11-08:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 624.378, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2497.512, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3121.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3121.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2497.512, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053", + "resource": { + "resourceType": "Encounter", + "id": "4f4d4b57-93d6-4990-847d-024cdf717053", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "210098006", + "display": "Domiciliary or rest home patient evaluation and management" + } + ], + "text": "Domiciliary or rest home patient evaluation and management" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2013-02-02T20:05:11-08:00", + "end": "2013-02-02T20:35:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0ab0ed89-1c8f-4fd0-ac1a-17e99365452f", + "resource": { + "resourceType": "Observation", + "id": "0ab0ed89-1c8f-4fd0-ac1a-17e99365452f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "76690-7", + "display": "Sexual orientation" + } + ], + "text": "Sexual orientation" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueString": "heterosexual" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5daf3999-0556-463c-a235-5777e4e45de3", + "resource": { + "resourceType": "Observation", + "id": "5daf3999-0556-463c-a235-5777e4e45de3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "55277-8", + "display": "HIV status" + } + ], + "text": "HIV status" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueString": "not HIV positive" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3a1a780-bcdc-494a-86c2-958549066741", + "resource": { + "resourceType": "Observation", + "id": "e3a1a780-bcdc-494a-86c2-958549066741", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "28245-9", + "display": "Abuse Status [OMAHA]" + } + ], + "text": "Abuse Status [OMAHA]" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueString": "No signs/symptoms" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:417916ff-44af-45a1-a6de-007ebc647401", + "resource": { + "resourceType": "Observation", + "id": "417916ff-44af-45a1-a6de-007ebc647401", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "71802-3", + "display": "Housing status" + } + ], + "text": "Housing status" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueString": "Patient is homeless" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d87effd-d8fa-4f89-bcb5-58a46d580e0e", + "resource": { + "resourceType": "Observation", + "id": "1d87effd-d8fa-4f89-bcb5-58a46d580e0e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "63513-6", + "display": "Are you covered by health insurance or some other kind of health care plan [PhenX]" + } + ], + "text": "Are you covered by health insurance or some other kind of health care plan [PhenX]" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueString": "No" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:78abbc38-83be-4428-a1cf-3a3caefac8f6", + "resource": { + "resourceType": "Observation", + "id": "78abbc38-83be-4428-a1cf-3a3caefac8f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "46240-8", + "display": "History of Hospitalizations+Outpatient visits" + } + ], + "text": "History of Hospitalizations+Outpatient visits" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "effectiveDateTime": "2013-02-02T20:05:11-08:00", + "issued": "2013-02-02T20:05:11.927-08:00", + "valueQuantity": { + "value": 5.7324, + "unit": "{count}", + "system": "http://unitsofmeasure.org", + "code": "{count}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:21081419-0b09-436f-98be-2307d0a81817", + "resource": { + "resourceType": "Procedure", + "id": "21081419-0b09-436f-98be-2307d0a81817", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "311791003", + "display": "Information gathering (procedure)" + } + ], + "text": "Information gathering (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + }, + "performedPeriod": { + "start": "2013-02-02T20:05:11-08:00", + "end": "2013-02-02T20:20:11-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:32027c54-5e42-495a-9c78-8888ebbc35c1", + "resource": { + "resourceType": "Claim", + "id": "32027c54-5e42-495a-9c78-8888ebbc35c1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-02-02T20:05:11-08:00", + "end": "2013-02-02T20:35:11-08:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:21081419-0b09-436f-98be-2307d0a81817" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "311791003" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:69381a58-a4f5-4546-a870-724c684559a4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "69381a58-a4f5-4546-a870-724c684559a4", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "32027c54-5e42-495a-9c78-8888ebbc35c1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-02-02T20:35:11-08:00", + "end": "2014-02-02T20:35:11-08:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "311791003" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034", + "resource": { + "resourceType": "Encounter", + "id": "3d4ef3f2-e300-44f8-a47a-e0cc63d47034", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "210098006", + "display": "Domiciliary or rest home patient evaluation and management" + } + ], + "text": "Domiciliary or rest home patient evaluation and management" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2013-03-04T20:05:11-08:00", + "end": "2013-03-04T20:35:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e520cd59-65d9-4e5d-8c4f-2d80ec5953f3", + "resource": { + "resourceType": "Observation", + "id": "e520cd59-65d9-4e5d-8c4f-2d80ec5953f3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "76690-7", + "display": "Sexual orientation" + } + ], + "text": "Sexual orientation" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueString": "heterosexual" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:569ed44c-1187-464c-82a0-da5d39deb5c3", + "resource": { + "resourceType": "Observation", + "id": "569ed44c-1187-464c-82a0-da5d39deb5c3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "55277-8", + "display": "HIV status" + } + ], + "text": "HIV status" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueString": "not HIV positive" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77adb897-9880-4377-953e-c76cd48d7b71", + "resource": { + "resourceType": "Observation", + "id": "77adb897-9880-4377-953e-c76cd48d7b71", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "28245-9", + "display": "Abuse Status [OMAHA]" + } + ], + "text": "Abuse Status [OMAHA]" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueString": "No signs/symptoms" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a7104ede-77ae-4616-9536-f8b23fe2bd10", + "resource": { + "resourceType": "Observation", + "id": "a7104ede-77ae-4616-9536-f8b23fe2bd10", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "71802-3", + "display": "Housing status" + } + ], + "text": "Housing status" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueString": "Patient is homeless" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:89851b54-0acc-4ea9-80f3-e01f9491151b", + "resource": { + "resourceType": "Observation", + "id": "89851b54-0acc-4ea9-80f3-e01f9491151b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "63513-6", + "display": "Are you covered by health insurance or some other kind of health care plan [PhenX]" + } + ], + "text": "Are you covered by health insurance or some other kind of health care plan [PhenX]" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueString": "No" + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ca3e3fa-c66b-4357-a0b9-1c3171d3f8ad", + "resource": { + "resourceType": "Observation", + "id": "9ca3e3fa-c66b-4357-a0b9-1c3171d3f8ad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "social-history", + "display": "social-history" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "46240-8", + "display": "History of Hospitalizations+Outpatient visits" + } + ], + "text": "History of Hospitalizations+Outpatient visits" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "effectiveDateTime": "2013-03-04T20:05:11-08:00", + "issued": "2013-03-04T20:05:11.927-08:00", + "valueQuantity": { + "value": 8.6061, + "unit": "{count}", + "system": "http://unitsofmeasure.org", + "code": "{count}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6", + "resource": { + "resourceType": "Procedure", + "id": "6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "311791003", + "display": "Information gathering (procedure)" + } + ], + "text": "Information gathering (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + }, + "performedPeriod": { + "start": "2013-03-04T20:05:11-08:00", + "end": "2013-03-04T20:20:11-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4303b920-9215-4025-b38f-44220da09e24", + "resource": { + "resourceType": "Claim", + "id": "4303b920-9215-4025-b38f-44220da09e24", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-03-04T20:05:11-08:00", + "end": "2013-03-04T20:35:11-08:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "311791003" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:28831e19-70c7-41e4-affa-58be1b22c66a", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "28831e19-70c7-41e4-affa-58be1b22c66a", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4303b920-9215-4025-b38f-44220da09e24" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-03-04T20:35:11-08:00", + "end": "2014-03-04T20:35:11-08:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "311791003" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443", + "resource": { + "resourceType": "Encounter", + "id": "1ac04561-97d1-4983-b0bc-3464237c7443", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2013-03-13T21:05:11-07:00", + "end": "2013-03-13T21:20:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:07bfee88-8db5-4fa7-90a6-01638ee9e6d1", + "resource": { + "resourceType": "Observation", + "id": "07bfee88-8db5-4fa7-90a6-01638ee9e6d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 76, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12943f61-67a0-4425-be25-9d02b7243f2f", + "resource": { + "resourceType": "Observation", + "id": "12943f61-67a0-4425-be25-9d02b7243f2f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0156d6ff-3378-4971-a882-8b8bb2aa5348", + "resource": { + "resourceType": "Observation", + "id": "0156d6ff-3378-4971-a882-8b8bb2aa5348", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 10.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:434e13bd-8671-48bb-bb9a-53a9d6ff5781", + "resource": { + "resourceType": "Observation", + "id": "434e13bd-8671-48bb-bb9a-53a9d6ff5781", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 61.221, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8ad5932-ae64-458c-8554-b0dd95fd11a6", + "resource": { + "resourceType": "Observation", + "id": "d8ad5932-ae64-458c-8554-b0dd95fd11a6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 45.990, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5eda79fe-29aa-494b-a5d0-1178a1e419d1", + "resource": { + "resourceType": "Observation", + "id": "5eda79fe-29aa-494b-a5d0-1178a1e419d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 88, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f4e2d59d-0c48-47d7-8ea2-aa5812897c69", + "resource": { + "resourceType": "Observation", + "id": "f4e2d59d-0c48-47d7-8ea2-aa5812897c69", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 73, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7b8c59c9-eb6b-4552-808e-033e23182ded", + "resource": { + "resourceType": "Observation", + "id": "7b8c59c9-eb6b-4552-808e-033e23182ded", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:23039a4c-fb0d-4d95-9ebd-6050f6fee708", + "resource": { + "resourceType": "Observation", + "id": "23039a4c-fb0d-4d95-9ebd-6050f6fee708", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "effectiveDateTime": "2013-03-13T21:05:11-07:00", + "issued": "2013-03-13T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cb2ea76-d545-431a-a26b-08691dcf6ab1", + "resource": { + "resourceType": "Immunization", + "id": "0cb2ea76-d545-431a-a26b-08691dcf6ab1", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "49", + "display": "Hib (PRP-OMP)" + } + ], + "text": "Hib (PRP-OMP)" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "date": "2013-03-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c", + "resource": { + "resourceType": "Immunization", + "id": "a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "date": "2013-03-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c", + "resource": { + "resourceType": "Immunization", + "id": "5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "date": "2013-03-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e581b310-2155-4126-89b9-8bc9a79483df", + "resource": { + "resourceType": "Immunization", + "id": "e581b310-2155-4126-89b9-8bc9a79483df", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "date": "2013-03-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:49f5a02d-bda1-4fd6-a46c-8e438a131ac0", + "resource": { + "resourceType": "Immunization", + "id": "49f5a02d-bda1-4fd6-a46c-8e438a131ac0", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + }, + "date": "2013-03-13T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14", + "resource": { + "resourceType": "Claim", + "id": "1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-03-13T21:05:11-07:00", + "end": "2013-03-13T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:0cb2ea76-d545-431a-a26b-08691dcf6ab1" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e581b310-2155-4126-89b9-8bc9a79483df" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:49f5a02d-bda1-4fd6-a46c-8e438a131ac0" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:739688b2-d627-48d0-aad8-d7bb600b407d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "739688b2-d627-48d0-aad8-d7bb600b407d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-03-13T21:20:11-07:00", + "end": "2014-03-13T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629", + "resource": { + "resourceType": "Encounter", + "id": "04a40a21-1488-4fab-a0f0-832765c91629", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4", + "resource": { + "resourceType": "Condition", + "id": "cf4fa5f3-360e-4adf-8dbe-48724beac3b4", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + }, + "onsetDateTime": "2013-05-12T21:05:11-07:00", + "abatementDateTime": "2013-06-12T21:05:11-07:00", + "assertedDate": "2013-05-12T21:05:11-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ff4c2f70-0740-44d9-bdcf-9a413399b902", + "resource": { + "resourceType": "MedicationRequest", + "id": "ff4c2f70-0740-44d9-bdcf-9a413399b902", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308182", + "display": "Amoxicillin 250 MG Oral Capsule" + } + ], + "text": "Amoxicillin 250 MG Oral Capsule" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + }, + "authoredOn": "2013-05-12T21:05:11-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:768f2ce4-4906-43ec-b75e-4539acb28c38", + "resource": { + "resourceType": "Claim", + "id": "768f2ce4-4906-43ec-b75e-4539acb28c38", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:ff4c2f70-0740-44d9-bdcf-9a413399b902" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + } + ] + } + ], + "total": { + "value": 4.11, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a7858988-a2d3-425d-871e-50e8ba7fd234", + "resource": { + "resourceType": "MedicationRequest", + "id": "a7858988-a2d3-425d-871e-50e8ba7fd234", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + }, + "authoredOn": "2013-05-12T21:05:11-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:8f93911c-303b-4c1a-8660-3a4ceaa7c16e", + "resource": { + "resourceType": "Claim", + "id": "8f93911c-303b-4c1a-8660-3a4ceaa7c16e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:a7858988-a2d3-425d-871e-50e8ba7fd234" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + } + ] + } + ], + "total": { + "value": 3.24, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ba298548-2f1c-4050-8f2d-01f8a0dce4b2", + "resource": { + "resourceType": "Claim", + "id": "ba298548-2f1c-4050-8f2d-01f8a0dce4b2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5fc4068c-ceb6-4b2c-9923-e6c54e7c22b5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5fc4068c-ceb6-4b2c-9923-e6c54e7c22b5", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ba298548-2f1c-4050-8f2d-01f8a0dce4b2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-05-12T21:20:11-07:00", + "end": "2014-05-12T21:20:11-07:00" + }, + "created": "2013-05-12T21:20:11-07:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:ba298548-2f1c-4050-8f2d-01f8a0dce4b2" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2013-05-12T21:05:11-07:00", + "end": "2013-05-12T21:20:11-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d", + "resource": { + "resourceType": "Encounter", + "id": "630b6bc9-4bce-421d-b42b-ced0b61bfc1d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2013-06-12T21:05:11-07:00", + "end": "2013-06-12T21:20:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:16c1366e-020a-4683-b698-8aeb6b4f9de9", + "resource": { + "resourceType": "Observation", + "id": "16c1366e-020a-4683-b698-8aeb6b4f9de9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 78.900, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:43c11759-e774-4878-a48d-c2e6b8bc171d", + "resource": { + "resourceType": "Observation", + "id": "43c11759-e774-4878-a48d-c2e6b8bc171d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:238cfd3f-a55e-4221-bddf-6375dc0ae15c", + "resource": { + "resourceType": "Observation", + "id": "238cfd3f-a55e-4221-bddf-6375dc0ae15c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 10.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:93bde22e-3692-4bef-8ee4-1d603ab44e2d", + "resource": { + "resourceType": "Observation", + "id": "93bde22e-3692-4bef-8ee4-1d603ab44e2d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 56.580, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eb3c872c-da8a-4dca-a2f1-48c56ea136ef", + "resource": { + "resourceType": "Observation", + "id": "eb3c872c-da8a-4dca-a2f1-48c56ea136ef", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 46.610, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:87cf9b73-8553-4570-be0d-92e3a07c3664", + "resource": { + "resourceType": "Observation", + "id": "87cf9b73-8553-4570-be0d-92e3a07c3664", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bb09f1d-f929-44c2-b883-366f77580380", + "resource": { + "resourceType": "Observation", + "id": "7bb09f1d-f929-44c2-b883-366f77580380", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 71, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b4f036b-9727-4701-816e-f0c9c8971102", + "resource": { + "resourceType": "Observation", + "id": "8b4f036b-9727-4701-816e-f0c9c8971102", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7dc653f0-d600-486a-8f6d-28e375dceb92", + "resource": { + "resourceType": "Observation", + "id": "7dc653f0-d600-486a-8f6d-28e375dceb92", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "effectiveDateTime": "2013-06-12T21:05:11-07:00", + "issued": "2013-06-12T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d87f5df-cbc2-4520-b461-ae3f947f84b3", + "resource": { + "resourceType": "Immunization", + "id": "6d87f5df-cbc2-4520-b461-ae3f947f84b3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + }, + "date": "2013-06-12T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:0d0d8ca4-0f73-4770-8f20-fa13d3bafa31", + "resource": { + "resourceType": "Claim", + "id": "0d0d8ca4-0f73-4770-8f20-fa13d3bafa31", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-06-12T21:05:11-07:00", + "end": "2013-06-12T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:6d87f5df-cbc2-4520-b461-ae3f947f84b3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a4960828-5830-4f8f-94a8-950cb4c3278c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a4960828-5830-4f8f-94a8-950cb4c3278c", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0d0d8ca4-0f73-4770-8f20-fa13d3bafa31" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-06-12T21:20:11-07:00", + "end": "2014-06-12T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220", + "resource": { + "resourceType": "Encounter", + "id": "e6e37c5f-169c-4541-9398-654855632220", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2013-12-11T20:05:11-08:00", + "end": "2013-12-11T20:35:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:710bb7ee-7de5-4d47-baf1-49fb0111def1", + "resource": { + "resourceType": "Observation", + "id": "710bb7ee-7de5-4d47-baf1-49fb0111def1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 84, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c9397236-bb65-4d44-a78e-ec4543b323e6", + "resource": { + "resourceType": "Observation", + "id": "c9397236-bb65-4d44-a78e-ec4543b323e6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ca6345d4-576b-413d-b695-245e95aebf7d", + "resource": { + "resourceType": "Observation", + "id": "ca6345d4-576b-413d-b695-245e95aebf7d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 11.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36d1f125-237a-4fbc-becf-116c878ee167", + "resource": { + "resourceType": "Observation", + "id": "36d1f125-237a-4fbc-becf-116c878ee167", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 50.402, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f5b04bb3-8fbe-4f97-99b2-afdb04369b7e", + "resource": { + "resourceType": "Observation", + "id": "f5b04bb3-8fbe-4f97-99b2-afdb04369b7e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 47.460, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6eb67837-fc64-436b-ad5f-30ff74eca795", + "resource": { + "resourceType": "Observation", + "id": "6eb67837-fc64-436b-ad5f-30ff74eca795", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df217223-0909-45d0-b6ef-8f211a25187d", + "resource": { + "resourceType": "Observation", + "id": "df217223-0909-45d0-b6ef-8f211a25187d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 89, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ffd13286-968f-48dd-ba24-6d8cfb2cfdb0", + "resource": { + "resourceType": "Observation", + "id": "ffd13286-968f-48dd-ba24-6d8cfb2cfdb0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:241cfb35-9d6f-4654-b76d-23256bdd6990", + "resource": { + "resourceType": "Observation", + "id": "241cfb35-9d6f-4654-b76d-23256bdd6990", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "effectiveDateTime": "2013-12-11T20:05:11-08:00", + "issued": "2013-12-11T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2c9bca50-2512-4511-b04f-a85aaa536f19", + "resource": { + "resourceType": "Procedure", + "id": "2c9bca50-2512-4511-b04f-a85aaa536f19", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "performedPeriod": { + "start": "2013-12-11T20:05:11-08:00", + "end": "2013-12-11T20:20:11-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7f811192-0c03-4448-a371-f349b14e0e0d", + "resource": { + "resourceType": "Immunization", + "id": "7f811192-0c03-4448-a371-f349b14e0e0d", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + }, + "date": "2013-12-11T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1c023d1e-1c27-47aa-8af8-4e40a119f9f6", + "resource": { + "resourceType": "Claim", + "id": "1c023d1e-1c27-47aa-8af8-4e40a119f9f6", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2013-12-11T20:05:11-08:00", + "end": "2013-12-11T20:35:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7f811192-0c03-4448-a371-f349b14e0e0d" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:2c9bca50-2512-4511-b04f-a85aaa536f19" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 788.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:15f60b34-ddf7-41ef-affa-73a0b93e40cf", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "15f60b34-ddf7-41ef-affa-73a0b93e40cf", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1c023d1e-1c27-47aa-8af8-4e40a119f9f6" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-12-11T20:35:11-08:00", + "end": "2014-12-11T20:35:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 157.686, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 630.744, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 788.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 788.43, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 743.1600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba", + "resource": { + "resourceType": "Encounter", + "id": "3176da2e-1231-4ba8-95e6-41e5c9de93ba", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2014-06-11T21:05:11-07:00", + "end": "2014-06-11T21:35:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9137e6df-7c09-477b-87ef-e017d1f4f45e", + "resource": { + "resourceType": "Observation", + "id": "9137e6df-7c09-477b-87ef-e017d1f4f45e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 87.700, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0aeab6bc-740e-4e22-ad25-450dd5e3eb9c", + "resource": { + "resourceType": "Observation", + "id": "0aeab6bc-740e-4e22-ad25-450dd5e3eb9c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:34c35f67-19be-4c3d-958e-d5ece010b4c6", + "resource": { + "resourceType": "Observation", + "id": "34c35f67-19be-4c3d-958e-d5ece010b4c6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 12.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1bcfb693-e2ab-4ea7-8687-b3953fa078e2", + "resource": { + "resourceType": "Observation", + "id": "1bcfb693-e2ab-4ea7-8687-b3953fa078e2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 37.678, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f3bdfcc7-9d23-46e8-9d6a-f1a6c867666d", + "resource": { + "resourceType": "Observation", + "id": "f3bdfcc7-9d23-46e8-9d6a-f1a6c867666d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 47.990, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d52542af-7a4a-4a1c-a332-6052018ed37e", + "resource": { + "resourceType": "Observation", + "id": "d52542af-7a4a-4a1c-a332-6052018ed37e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 16.160, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c300b10-ecff-4296-a8c0-4bd15efbf4d1", + "resource": { + "resourceType": "Observation", + "id": "0c300b10-ecff-4296-a8c0-4bd15efbf4d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 45.562, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cca9da08-699c-4152-89a4-7cae0ed7a1fe", + "resource": { + "resourceType": "Observation", + "id": "cca9da08-699c-4152-89a4-7cae0ed7a1fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 104, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9dc6d8c7-c78d-4202-ab0a-99d32f1795c8", + "resource": { + "resourceType": "Observation", + "id": "9dc6d8c7-c78d-4202-ab0a-99d32f1795c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 62, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:58a8ff38-e132-49ed-88aa-f670ddc47063", + "resource": { + "resourceType": "Observation", + "id": "58a8ff38-e132-49ed-88aa-f670ddc47063", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:acad2f45-f255-4cc1-a376-f3ceb0ce414a", + "resource": { + "resourceType": "Observation", + "id": "acad2f45-f255-4cc1-a376-f3ceb0ce414a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "effectiveDateTime": "2014-06-11T21:05:11-07:00", + "issued": "2014-06-11T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:51eae485-2625-40aa-8b20-9e1d0e1e61f9", + "resource": { + "resourceType": "Procedure", + "id": "51eae485-2625-40aa-8b20-9e1d0e1e61f9", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "performedPeriod": { + "start": "2014-06-11T21:05:11-07:00", + "end": "2014-06-11T21:20:11-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c3e287d7-8bb8-4c97-b719-0cb0fc3f650f", + "resource": { + "resourceType": "Immunization", + "id": "c3e287d7-8bb8-4c97-b719-0cb0fc3f650f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + }, + "date": "2014-06-11T21:05:11-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e", + "resource": { + "resourceType": "Claim", + "id": "113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-06-11T21:05:11-07:00", + "end": "2014-06-11T21:35:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c3e287d7-8bb8-4c97-b719-0cb0fc3f650f" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:51eae485-2625-40aa-8b20-9e1d0e1e61f9" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 898.40, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:967b251b-ee1c-491d-ac21-3d21df8df8a9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "967b251b-ee1c-491d-ac21-3d21df8df8a9", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-06-11T21:35:11-07:00", + "end": "2015-06-11T21:35:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 179.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 718.72, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 898.40, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 898.40, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 831.1360000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357", + "resource": { + "resourceType": "Encounter", + "id": "f07a921a-582d-437e-b19e-46d8d6da8357", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0", + "resource": { + "resourceType": "Condition", + "id": "74e3b413-1939-43e9-b43d-782a1f5b13f0", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + }, + "onsetDateTime": "2014-11-04T20:05:11-08:00", + "abatementDateTime": "2014-12-10T20:05:11-08:00", + "assertedDate": "2014-11-04T20:05:11-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ea6e0ecc-fbbd-4bad-ba83-4875393c71fd", + "resource": { + "resourceType": "MedicationRequest", + "id": "ea6e0ecc-fbbd-4bad-ba83-4875393c71fd", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308182", + "display": "Amoxicillin 250 MG Oral Capsule" + } + ], + "text": "Amoxicillin 250 MG Oral Capsule" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + }, + "authoredOn": "2014-11-04T20:05:11-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:bddac4ad-576b-4027-9324-75f21ffcba27", + "resource": { + "resourceType": "Claim", + "id": "bddac4ad-576b-4027-9324-75f21ffcba27", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:ea6e0ecc-fbbd-4bad-ba83-4875393c71fd" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + } + ] + } + ], + "total": { + "value": 8.05, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c35e09f3-b451-4765-b703-d2dc13d7b340", + "resource": { + "resourceType": "MedicationRequest", + "id": "c35e09f3-b451-4765-b703-d2dc13d7b340", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "198405", + "display": "Ibuprofen 100 MG Oral Tablet" + } + ], + "text": "Ibuprofen 100 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + }, + "authoredOn": "2014-11-04T20:05:11-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1e02d38a-86b1-4ce9-a848-fba736c1ec7d", + "resource": { + "resourceType": "Claim", + "id": "1e02d38a-86b1-4ce9-a848-fba736c1ec7d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:c35e09f3-b451-4765-b703-d2dc13d7b340" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + } + ] + } + ], + "total": { + "value": 37.1, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:91e1a143-5672-4954-a626-ddeb4a0b9974", + "resource": { + "resourceType": "Claim", + "id": "91e1a143-5672-4954-a626-ddeb4a0b9974", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:99de1846-3de5-4fc7-91ca-c6770cc86bee", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "99de1846-3de5-4fc7-91ca-c6770cc86bee", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "91e1a143-5672-4954-a626-ddeb4a0b9974" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-11-04T20:20:11-08:00", + "end": "2015-11-04T20:20:11-08:00" + }, + "created": "2014-11-04T20:20:11-08:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:91e1a143-5672-4954-a626-ddeb4a0b9974" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-11-04T20:05:11-08:00", + "end": "2014-11-04T20:20:11-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d", + "resource": { + "resourceType": "Encounter", + "id": "92c83229-bcec-4d68-bd71-d282d7f0052d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2014-12-10T20:05:11-08:00", + "end": "2014-12-10T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:56655e3a-4be3-4121-a5f9-a1b1ae7ee983", + "resource": { + "resourceType": "Observation", + "id": "56655e3a-4be3-4121-a5f9-a1b1ae7ee983", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 91.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:26ee22da-ae0a-4e47-a934-caa664265146", + "resource": { + "resourceType": "Observation", + "id": "26ee22da-ae0a-4e47-a934-caa664265146", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77c6092e-197f-4338-a216-bdea9f6598a0", + "resource": { + "resourceType": "Observation", + "id": "77c6092e-197f-4338-a216-bdea9f6598a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 12.700, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d72f431b-d180-4098-83ee-ac98bd314455", + "resource": { + "resourceType": "Observation", + "id": "d72f431b-d180-4098-83ee-ac98bd314455", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 17.050, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d08a9cfd-81d7-40b7-8565-2af0ce9ba012", + "resource": { + "resourceType": "Observation", + "id": "d08a9cfd-81d7-40b7-8565-2af0ce9ba012", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 48.320, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9fa7636a-dcca-4748-9421-c9a2e8fffaa6", + "resource": { + "resourceType": "Observation", + "id": "9fa7636a-dcca-4748-9421-c9a2e8fffaa6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 15.170, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9dc13a84-663c-49db-b494-b656d9d250a2", + "resource": { + "resourceType": "Observation", + "id": "9dc13a84-663c-49db-b494-b656d9d250a2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 21.242, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4e78f834-8a3c-4541-bb4c-b582e575f620", + "resource": { + "resourceType": "Observation", + "id": "4e78f834-8a3c-4541-bb4c-b582e575f620", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 78, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9f9ae521-9b30-4791-be08-698827a0d63d", + "resource": { + "resourceType": "Observation", + "id": "9f9ae521-9b30-4791-be08-698827a0d63d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 61, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:98c2b2c4-5917-42ab-8338-91dde415d58d", + "resource": { + "resourceType": "Observation", + "id": "98c2b2c4-5917-42ab-8338-91dde415d58d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2967b643-54cb-4608-806d-76c019076056", + "resource": { + "resourceType": "Observation", + "id": "2967b643-54cb-4608-806d-76c019076056", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "effectiveDateTime": "2014-12-10T20:05:11-08:00", + "issued": "2014-12-10T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50ed6852-2852-42a7-9ec8-c187270a8d1f", + "resource": { + "resourceType": "Immunization", + "id": "50ed6852-2852-42a7-9ec8-c187270a8d1f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + }, + "date": "2014-12-10T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2", + "resource": { + "resourceType": "Claim", + "id": "ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2014-12-10T20:05:11-08:00", + "end": "2014-12-10T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:50ed6852-2852-42a7-9ec8-c187270a8d1f" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8f1e1663-b4ce-46e8-aaf1-0b31cca0a9a0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8f1e1663-b4ce-46e8-aaf1-0b31cca0a9a0", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-12-10T20:20:11-08:00", + "end": "2015-12-10T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e", + "resource": { + "resourceType": "Encounter", + "id": "3a063f0c-80ce-479e-a6f1-01a6421a9f0e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2015-05-06T21:05:11-07:00", + "end": "2015-05-06T21:35:11-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e", + "resource": { + "resourceType": "Condition", + "id": "0ee4180b-3fcb-4850-a816-f745e3a6b79e", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" + }, + "onsetDateTime": "2015-05-06T21:05:11-07:00", + "abatementDateTime": "2015-05-15T21:05:11-07:00", + "assertedDate": "2015-05-06T21:05:11-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ecf6e70e-bdfe-4421-a48d-e257a5514ad6", + "resource": { + "resourceType": "Observation", + "id": "ecf6e70e-bdfe-4421-a48d-e257a5514ad6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" + }, + "effectiveDateTime": "2015-05-06T21:05:11-07:00", + "issued": "2015-05-06T21:05:11.927-07:00", + "valueQuantity": { + "value": 37.538, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4a1db053-bc05-452b-8f91-0b1ea35a94eb", + "resource": { + "resourceType": "Procedure", + "id": "4a1db053-bc05-452b-8f91-0b1ea35a94eb", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" + }, + "performedPeriod": { + "start": "2015-05-06T21:05:11-07:00", + "end": "2015-05-06T21:20:11-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", + "display": "Acute viral pharyngitis (disorder)" + }, + { + "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e", + "display": "Acute viral pharyngitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:ae5b845e-062b-40d9-a4ee-65db59760c8d", + "resource": { + "resourceType": "Claim", + "id": "ae5b845e-062b-40d9-a4ee-65db59760c8d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2015-05-06T21:05:11-07:00", + "end": "2015-05-06T21:35:11-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4a1db053-bc05-452b-8f91-0b1ea35a94eb" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "net": { + "value": 2578.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c2ba8d5d-514f-45ef-92fb-7ebdb37de26b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c2ba8d5d-514f-45ef-92fb-7ebdb37de26b", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ae5b845e-062b-40d9-a4ee-65db59760c8d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-05-06T21:35:11-07:00", + "end": "2016-05-06T21:35:11-07:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 515.616, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2062.464, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2578.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2578.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2062.464, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe", + "resource": { + "resourceType": "Encounter", + "id": "a1d5506d-6503-44e6-8aa0-73c86fb538fe", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2015-06-10T21:05:11-07:00", + "end": "2015-06-10T21:35:11-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:20e18781-ca54-41a6-be68-fee4f84bc20e", + "resource": { + "resourceType": "Observation", + "id": "20e18781-ca54-41a6-be68-fee4f84bc20e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 95.200, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d217e7a9-30af-4bc0-be0a-0829b53c748d", + "resource": { + "resourceType": "Observation", + "id": "d217e7a9-30af-4bc0-be0a-0829b53c748d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9397dd21-32d9-44bb-88f4-4f738ead8ca3", + "resource": { + "resourceType": "Observation", + "id": "9397dd21-32d9-44bb-88f4-4f738ead8ca3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 13.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:02bf437b-44b2-4b93-b3f3-985fdea74f33", + "resource": { + "resourceType": "Observation", + "id": "02bf437b-44b2-4b93-b3f3-985fdea74f33", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 19.973, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df1d7890-006b-4dc0-a043-cf88ef538f09", + "resource": { + "resourceType": "Observation", + "id": "df1d7890-006b-4dc0-a043-cf88ef538f09", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 48.340, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4586c508-3e96-45e5-bd88-410e97ee96ea", + "resource": { + "resourceType": "Observation", + "id": "4586c508-3e96-45e5-bd88-410e97ee96ea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 14.520, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e10536a-5aec-4765-9807-e61ec16e5b60", + "resource": { + "resourceType": "Observation", + "id": "8e10536a-5aec-4765-9807-e61ec16e5b60", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 10.006, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a238fa6-4fae-4381-81e4-820f9cca4cb9", + "resource": { + "resourceType": "Observation", + "id": "8a238fa6-4fae-4381-81e4-820f9cca4cb9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 124, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:26601664-5b83-4000-a4d3-d5079556dc57", + "resource": { + "resourceType": "Observation", + "id": "26601664-5b83-4000-a4d3-d5079556dc57", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 62, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1c82fc69-c853-4366-ae7a-a1008fc02265", + "resource": { + "resourceType": "Observation", + "id": "1c82fc69-c853-4366-ae7a-a1008fc02265", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e2fb7236-49fa-42cc-9833-9d16aab4d918", + "resource": { + "resourceType": "Observation", + "id": "e2fb7236-49fa-42cc-9833-9d16aab4d918", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "effectiveDateTime": "2015-06-10T21:05:11-07:00", + "issued": "2015-06-10T21:05:11.927-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cf24ed92-3dc9-4bc0-b850-b252f0e7d200", + "resource": { + "resourceType": "Procedure", + "id": "cf24ed92-3dc9-4bc0-b850-b252f0e7d200", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + }, + "performedPeriod": { + "start": "2015-06-10T21:05:11-07:00", + "end": "2015-06-10T21:20:11-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:45983547-9f85-44d3-aaaf-60897a1fe192", + "resource": { + "resourceType": "Claim", + "id": "45983547-9f85-44d3-aaaf-60897a1fe192", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2015-06-10T21:05:11-07:00", + "end": "2015-06-10T21:35:11-07:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:cf24ed92-3dc9-4bc0-b850-b252f0e7d200" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 874.29, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d7deff37-292b-46b6-aa2f-544798e0966e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d7deff37-292b-46b6-aa2f-544798e0966e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "45983547-9f85-44d3-aaaf-60897a1fe192" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-06-10T21:35:11-07:00", + "end": "2016-06-10T21:35:11-07:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 174.858, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 699.432, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 874.29, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 874.29, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 699.432, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd", + "resource": { + "resourceType": "Encounter", + "id": "636f051c-bece-4ff2-b60a-a896bd3c11cd", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2015-09-20T21:05:11-07:00", + "end": "2015-09-20T21:20:11-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6", + "resource": { + "resourceType": "Condition", + "id": "03a85dd0-bd02-4318-ac7f-33d0b30903b6", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd" + }, + "onsetDateTime": "2015-09-20T21:05:11-07:00", + "abatementDateTime": "2015-10-04T21:05:11-07:00", + "assertedDate": "2015-09-20T21:05:11-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:e45a1099-63af-4d04-b1dd-fc0a4e6bbab6", + "resource": { + "resourceType": "Claim", + "id": "e45a1099-63af-4d04-b1dd-fc0a4e6bbab6", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2015-09-20T21:05:11-07:00", + "end": "2015-09-20T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7322204f-9eb4-4c12-9082-8ec045f053cd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7322204f-9eb4-4c12-9082-8ec045f053cd", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e45a1099-63af-4d04-b1dd-fc0a4e6bbab6" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-09-20T21:20:11-07:00", + "end": "2016-09-20T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629", + "resource": { + "resourceType": "Encounter", + "id": "67c12bfb-95a8-42c6-9340-a15cc1a55629", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2015-12-09T20:05:11-08:00", + "end": "2015-12-09T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:520c948e-4089-4883-a104-f5764e8964f2", + "resource": { + "resourceType": "Observation", + "id": "520c948e-4089-4883-a104-f5764e8964f2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 98.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36eb9e11-5780-43ab-b1c7-b0dcaf821ca4", + "resource": { + "resourceType": "Observation", + "id": "36eb9e11-5780-43ab-b1c7-b0dcaf821ca4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:64a8ed52-5e90-4906-81c8-cca235a848c6", + "resource": { + "resourceType": "Observation", + "id": "64a8ed52-5e90-4906-81c8-cca235a848c6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 14.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b70bd543-daad-4fdc-9b6b-6528add2fc8c", + "resource": { + "resourceType": "Observation", + "id": "b70bd543-daad-4fdc-9b6b-6528add2fc8c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 19.973, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6078066b-daea-4b15-a319-eea35bd76573", + "resource": { + "resourceType": "Observation", + "id": "6078066b-daea-4b15-a319-eea35bd76573", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 48.340, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dbf72292-37e0-4c94-b475-6d1c9c1af60f", + "resource": { + "resourceType": "Observation", + "id": "dbf72292-37e0-4c94-b475-6d1c9c1af60f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 14.730, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7ada8175-86e1-471a-a361-f6a23bb332cf", + "resource": { + "resourceType": "Observation", + "id": "7ada8175-86e1-471a-a361-f6a23bb332cf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 18.646, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6cc1395e-0f96-4ca5-bcf8-a71957ed8f24", + "resource": { + "resourceType": "Observation", + "id": "6cc1395e-0f96-4ca5-bcf8-a71957ed8f24", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 117, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:44bef88f-f1f9-4702-b063-54ef033a4307", + "resource": { + "resourceType": "Observation", + "id": "44bef88f-f1f9-4702-b063-54ef033a4307", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 76, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8b348046-2f08-446d-80c0-e982b83637bc", + "resource": { + "resourceType": "Observation", + "id": "8b348046-2f08-446d-80c0-e982b83637bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:539a3c6d-5f18-4225-a839-a47f8a0e0c68", + "resource": { + "resourceType": "Observation", + "id": "539a3c6d-5f18-4225-a839-a47f8a0e0c68", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "effectiveDateTime": "2015-12-09T20:05:11-08:00", + "issued": "2015-12-09T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4259fe9b-5bfe-4be4-a8b7-707e44be6dde", + "resource": { + "resourceType": "Immunization", + "id": "4259fe9b-5bfe-4be4-a8b7-707e44be6dde", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + }, + "date": "2015-12-09T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d640d081-ad7e-408b-b692-6c32c7c53864", + "resource": { + "resourceType": "Claim", + "id": "d640d081-ad7e-408b-b692-6c32c7c53864", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2015-12-09T20:05:11-08:00", + "end": "2015-12-09T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4259fe9b-5bfe-4be4-a8b7-707e44be6dde" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3275cdec-5461-4170-be46-6efc703d0e5c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3275cdec-5461-4170-be46-6efc703d0e5c", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d640d081-ad7e-408b-b692-6c32c7c53864" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-12-09T20:20:11-08:00", + "end": "2016-12-09T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810", + "resource": { + "resourceType": "Encounter", + "id": "3bd80262-f307-4c18-8198-72d23d3bf810", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + } + ], + "period": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65", + "resource": { + "resourceType": "Condition", + "id": "aa9df8bf-c94c-40a2-8483-fcfab7698c65", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + }, + "onsetDateTime": "2016-08-01T21:05:11-07:00", + "abatementDateTime": "2016-12-14T20:05:11-08:00", + "assertedDate": "2016-08-01T21:05:11-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:326cc4c9-c367-467b-9698-1279efd7f02a", + "resource": { + "resourceType": "MedicationRequest", + "id": "326cc4c9-c367-467b-9698-1279efd7f02a", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308182", + "display": "Amoxicillin 250 MG Oral Capsule" + } + ], + "text": "Amoxicillin 250 MG Oral Capsule" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + }, + "authoredOn": "2016-08-01T21:05:11-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:7e1318f4-6d5e-4ac3-a33f-6435543c9833", + "resource": { + "resourceType": "Claim", + "id": "7e1318f4-6d5e-4ac3-a33f-6435543c9833", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:326cc4c9-c367-467b-9698-1279efd7f02a" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + } + ] + } + ], + "total": { + "value": 4.61, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a9493add-b2e9-46f9-bc58-6502eb98b460", + "resource": { + "resourceType": "MedicationRequest", + "id": "a9493add-b2e9-46f9-bc58-6502eb98b460", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "198405", + "display": "Ibuprofen 100 MG Oral Tablet" + } + ], + "text": "Ibuprofen 100 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + }, + "authoredOn": "2016-08-01T21:05:11-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" + }, + "onBehalfOf": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:4d3ba025-4385-4353-80b4-b19e643c5c9c", + "resource": { + "resourceType": "Claim", + "id": "4d3ba025-4385-4353-80b4-b19e643c5c9c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "prescription": { + "reference": "urn:uuid:a9493add-b2e9-46f9-bc58-6502eb98b460" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + } + ] + } + ], + "total": { + "value": 11.34, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:50484348-f8e8-438b-a79b-136d3960eb53", + "resource": { + "resourceType": "Claim", + "id": "50484348-f8e8-438b-a79b-136d3960eb53", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4c08e570-9d65-452f-a2e3-4f6784cfcf11", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4c08e570-9d65-452f-a2e3-4f6784cfcf11", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "50484348-f8e8-438b-a79b-136d3960eb53" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2016-08-01T21:20:11-07:00", + "end": "2017-08-01T21:20:11-07:00" + }, + "created": "2016-08-01T21:20:11-07:00", + "provider": { + "identifier": { + "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" + } + }, + "organization": { + "identifier": { + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:50484348-f8e8-438b-a79b-136d3960eb53" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-08-01T21:05:11-07:00", + "end": "2016-08-01T21:20:11-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431", + "resource": { + "resourceType": "Encounter", + "id": "32e3dc55-9e43-454a-b8bd-8636caed2431", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2016-12-14T20:05:11-08:00", + "end": "2016-12-14T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d4ae6708-d33f-4136-b1ec-fff488e3dfa6", + "resource": { + "resourceType": "Observation", + "id": "d4ae6708-d33f-4136-b1ec-fff488e3dfa6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 104.90, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd669a47-7e48-4fbc-bd5c-e64145f260bc", + "resource": { + "resourceType": "Observation", + "id": "cd669a47-7e48-4fbc-bd5c-e64145f260bc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f570a2d2-8fb3-4be4-9674-713995a72d6e", + "resource": { + "resourceType": "Observation", + "id": "f570a2d2-8fb3-4be4-9674-713995a72d6e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 16.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:338c794c-0240-421b-b8e8-854e6b3f6040", + "resource": { + "resourceType": "Observation", + "id": "338c794c-0240-421b-b8e8-854e6b3f6040", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 15.100, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:97ca8f54-1297-4a55-b251-0a44edf07957", + "resource": { + "resourceType": "Observation", + "id": "97ca8f54-1297-4a55-b251-0a44edf07957", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 38.379, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5c8f8ade-9882-486a-a65b-f8df5ef6d884", + "resource": { + "resourceType": "Observation", + "id": "5c8f8ade-9882-486a-a65b-f8df5ef6d884", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 118, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c502755-6247-4854-b958-6ac305837ce7", + "resource": { + "resourceType": "Observation", + "id": "7c502755-6247-4854-b958-6ac305837ce7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 92, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fabcc616-2fb9-4486-b7dc-b334292e1951", + "resource": { + "resourceType": "Observation", + "id": "fabcc616-2fb9-4486-b7dc-b334292e1951", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2ee4d489-1b7a-4a24-9a03-b8ca4813e965", + "resource": { + "resourceType": "Observation", + "id": "2ee4d489-1b7a-4a24-9a03-b8ca4813e965", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "effectiveDateTime": "2016-12-14T20:05:11-08:00", + "issued": "2016-12-14T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c46eaed1-1343-4862-9990-5f8f969bac48", + "resource": { + "resourceType": "Immunization", + "id": "c46eaed1-1343-4862-9990-5f8f969bac48", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "date": "2016-12-14T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:89be39a5-7a32-4e13-b5a8-914d56b69f44", + "resource": { + "resourceType": "Immunization", + "id": "89be39a5-7a32-4e13-b5a8-914d56b69f44", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "date": "2016-12-14T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:77fb8590-d460-4b5f-8729-c3f9ca2557c5", + "resource": { + "resourceType": "Immunization", + "id": "77fb8590-d460-4b5f-8729-c3f9ca2557c5", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "date": "2016-12-14T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:adae9219-f4b0-43bf-b48e-1c77d49ca55a", + "resource": { + "resourceType": "Immunization", + "id": "adae9219-f4b0-43bf-b48e-1c77d49ca55a", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "date": "2016-12-14T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:a41d2025-3ce6-48fa-9c88-60090d305cde", + "resource": { + "resourceType": "Immunization", + "id": "a41d2025-3ce6-48fa-9c88-60090d305cde", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + }, + "date": "2016-12-14T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8c2915cd-77a5-4b63-b6f4-d7461f1d761f", + "resource": { + "resourceType": "Claim", + "id": "8c2915cd-77a5-4b63-b6f4-d7461f1d761f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2016-12-14T20:05:11-08:00", + "end": "2016-12-14T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c46eaed1-1343-4862-9990-5f8f969bac48" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:89be39a5-7a32-4e13-b5a8-914d56b69f44" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:77fb8590-d460-4b5f-8729-c3f9ca2557c5" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:adae9219-f4b0-43bf-b48e-1c77d49ca55a" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:a41d2025-3ce6-48fa-9c88-60090d305cde" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:936fed72-50f6-4c0a-b60f-5d16b77c588f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "936fed72-50f6-4c0a-b60f-5d16b77c588f", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "8c2915cd-77a5-4b63-b6f4-d7461f1d761f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-12-14T20:20:11-08:00", + "end": "2017-12-14T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8", + "resource": { + "resourceType": "Encounter", + "id": "35981aed-9913-4e73-9dd0-3f60d0b33fb8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2017-12-20T20:05:11-08:00", + "end": "2017-12-20T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1bc1d55f-51d5-4f1f-be96-18967e7dbbdb", + "resource": { + "resourceType": "Observation", + "id": "1bc1d55f-51d5-4f1f-be96-18967e7dbbdb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 111.10, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b17a002f-243c-445a-b647-b9517a65bb0a", + "resource": { + "resourceType": "Observation", + "id": "b17a002f-243c-445a-b647-b9517a65bb0a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:38cbc1d5-1f2d-4418-9c45-494ed60b5d2b", + "resource": { + "resourceType": "Observation", + "id": "38cbc1d5-1f2d-4418-9c45-494ed60b5d2b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 19, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d123522c-61c4-48f6-9201-d786c56b56f6", + "resource": { + "resourceType": "Observation", + "id": "d123522c-61c4-48f6-9201-d786c56b56f6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 15.430, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:851d32e5-38c0-462c-b2fe-e5109360526f", + "resource": { + "resourceType": "Observation", + "id": "851d32e5-38c0-462c-b2fe-e5109360526f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 51.684, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:af271b9e-9056-4664-b0e3-d12788050fd1", + "resource": { + "resourceType": "Observation", + "id": "af271b9e-9056-4664-b0e3-d12788050fd1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1ae2627-9149-4100-93fb-2af4a2512b4b", + "resource": { + "resourceType": "Observation", + "id": "d1ae2627-9149-4100-93fb-2af4a2512b4b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 83, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a699695c-1228-4266-a933-2e33ecbb0f3a", + "resource": { + "resourceType": "Observation", + "id": "a699695c-1228-4266-a933-2e33ecbb0f3a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4ba136fd-edcb-4929-a1bd-95f8b7a60605", + "resource": { + "resourceType": "Observation", + "id": "4ba136fd-edcb-4929-a1bd-95f8b7a60605", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 10.106, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", + "resource": { + "resourceType": "Observation", + "id": "b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 5.4330, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:53a1d7e4-bcd1-490c-ba40-70f455b2cda8", + "resource": { + "resourceType": "Observation", + "id": "53a1d7e4-bcd1-490c-ba40-70f455b2cda8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 12.417, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9ee23889-a002-4b95-b887-cfd9233b62ad", + "resource": { + "resourceType": "Observation", + "id": "9ee23889-a002-4b95-b887-cfd9233b62ad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 44.338, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ddece709-c449-4ca4-85ff-755543487767", + "resource": { + "resourceType": "Observation", + "id": "ddece709-c449-4ca4-85ff-755543487767", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 93.280, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", + "resource": { + "resourceType": "Observation", + "id": "79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 31.997, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6d4da189-9300-49fc-ac5a-d709f8fa53f4", + "resource": { + "resourceType": "Observation", + "id": "6d4da189-9300-49fc-ac5a-d709f8fa53f4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 35.106, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:299455b2-3f69-4bce-acac-f170e788fd32", + "resource": { + "resourceType": "Observation", + "id": "299455b2-3f69-4bce-acac-f170e788fd32", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 39.407, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a69aa703-b799-4909-9488-e8b065f162c5", + "resource": { + "resourceType": "Observation", + "id": "a69aa703-b799-4909-9488-e8b065f162c5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 359.02, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebb6368f-d2ad-4b85-a279-3d49868b2105", + "resource": { + "resourceType": "Observation", + "id": "ebb6368f-d2ad-4b85-a279-3d49868b2105", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 325.39, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83aa45f2-ed79-4472-8481-f96de9e7b14a", + "resource": { + "resourceType": "Observation", + "id": "83aa45f2-ed79-4472-8481-f96de9e7b14a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueQuantity": { + "value": 9.4721, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a7cb848-3052-473f-a9f7-6b87f210bd34", + "resource": { + "resourceType": "Observation", + "id": "8a7cb848-3052-473f-a9f7-6b87f210bd34", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bdcec993-44b9-49ed-a9a5-46dc72a8e85b", + "resource": { + "resourceType": "Immunization", + "id": "bdcec993-44b9-49ed-a9a5-46dc72a8e85b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "date": "2017-12-20T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:50c78957-2341-4819-b93f-565dd19a828d", + "resource": { + "resourceType": "DiagnosticReport", + "id": "50c78957-2341-4819-b93f-565dd19a828d", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + }, + "effectiveDateTime": "2017-12-20T20:05:11-08:00", + "issued": "2017-12-20T20:05:11.927-08:00", + "result": [ + { + "reference": "urn:uuid:4ba136fd-edcb-4929-a1bd-95f8b7a60605", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:53a1d7e4-bcd1-490c-ba40-70f455b2cda8", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:9ee23889-a002-4b95-b887-cfd9233b62ad", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:ddece709-c449-4ca4-85ff-755543487767", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:6d4da189-9300-49fc-ac5a-d709f8fa53f4", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:299455b2-3f69-4bce-acac-f170e788fd32", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:a69aa703-b799-4909-9488-e8b065f162c5", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:ebb6368f-d2ad-4b85-a279-3d49868b2105", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:83aa45f2-ed79-4472-8481-f96de9e7b14a", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:f03e4ba9-9531-45ef-8e06-aae71d0f20b1", + "resource": { + "resourceType": "Claim", + "id": "f03e4ba9-9531-45ef-8e06-aae71d0f20b1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2017-12-20T20:05:11-08:00", + "end": "2017-12-20T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:bdcec993-44b9-49ed-a9a5-46dc72a8e85b" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a93c3d06-09db-4742-ba12-b11a89987343", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a93c3d06-09db-4742-ba12-b11a89987343", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f03e4ba9-9531-45ef-8e06-aae71d0f20b1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2017-12-20T20:20:11-08:00", + "end": "2018-12-20T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07", + "resource": { + "resourceType": "Encounter", + "id": "396ef73d-6551-43a5-a12a-97575e70ad07", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2018-03-27T21:05:11-07:00", + "end": "2018-03-27T21:20:11-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8", + "resource": { + "resourceType": "Condition", + "id": "fb53b877-fd5b-432a-80ff-6531ef2f9af8", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07" + }, + "onsetDateTime": "2018-03-27T21:05:11-07:00", + "abatementDateTime": "2018-04-10T21:05:11-07:00", + "assertedDate": "2018-03-27T21:05:11-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:67b9ab66-57a6-4ab0-917a-1240bea9d360", + "resource": { + "resourceType": "Claim", + "id": "67b9ab66-57a6-4ab0-917a-1240bea9d360", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2018-03-27T21:05:11-07:00", + "end": "2018-03-27T21:20:11-07:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b56c52d9-3800-4877-a1e0-df6a585cabce", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b56c52d9-3800-4877-a1e0-df6a585cabce", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "67b9ab66-57a6-4ab0-917a-1240bea9d360" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-03-27T21:20:11-07:00", + "end": "2019-03-27T21:20:11-07:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622", + "resource": { + "resourceType": "Encounter", + "id": "8a6adcfd-e483-46a7-ad84-dbce1ddd9622", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2018-12-26T20:05:11-08:00", + "end": "2018-12-26T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:47631bea-a001-4c5b-9e53-ad82a5da19b0", + "resource": { + "resourceType": "Observation", + "id": "47631bea-a001-4c5b-9e53-ad82a5da19b0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 117.20, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5e6538c6-eedf-4f4d-95b2-9164222fe96e", + "resource": { + "resourceType": "Observation", + "id": "5e6538c6-eedf-4f4d-95b2-9164222fe96e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fb80f3f7-e5c4-4409-8236-cf99a4f476ee", + "resource": { + "resourceType": "Observation", + "id": "fb80f3f7-e5c4-4409-8236-cf99a4f476ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 21.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:522fc591-692c-4966-9e67-dba157bd2556", + "resource": { + "resourceType": "Observation", + "id": "522fc591-692c-4966-9e67-dba157bd2556", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 15.670, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f22c7dd0-5c8f-4f93-8dd3-7baa3c3f9326", + "resource": { + "resourceType": "Observation", + "id": "f22c7dd0-5c8f-4f93-8dd3-7baa3c3f9326", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 54.696, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e08de7f1-0ccd-4ee3-8f10-71a5f2a115d6", + "resource": { + "resourceType": "Observation", + "id": "e08de7f1-0ccd-4ee3-8f10-71a5f2a115d6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 85, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c137e52e-d15a-4e1d-a96e-5b797f831553", + "resource": { + "resourceType": "Observation", + "id": "c137e52e-d15a-4e1d-a96e-5b797f831553", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 92, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:da03a0e0-5031-4f15-8e01-77a38ddcf812", + "resource": { + "resourceType": "Observation", + "id": "da03a0e0-5031-4f15-8e01-77a38ddcf812", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a2113e92-289c-4ea0-a3ee-8b745963ea5f", + "resource": { + "resourceType": "Observation", + "id": "a2113e92-289c-4ea0-a3ee-8b745963ea5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "effectiveDateTime": "2018-12-26T20:05:11-08:00", + "issued": "2018-12-26T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:42dd3bca-4dbe-4a9a-ac20-0878d46beb84", + "resource": { + "resourceType": "Immunization", + "id": "42dd3bca-4dbe-4a9a-ac20-0878d46beb84", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + }, + "date": "2018-12-26T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1", + "resource": { + "resourceType": "Claim", + "id": "d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2018-12-26T20:05:11-08:00", + "end": "2018-12-26T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:42dd3bca-4dbe-4a9a-ac20-0878d46beb84" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bf76dc6e-875f-4393-af68-704fe21afe0b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "bf76dc6e-875f-4393-af68-704fe21afe0b", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-12-26T20:20:11-08:00", + "end": "2019-12-26T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44", + "resource": { + "resourceType": "Encounter", + "id": "c233b803-3126-4dfa-a13a-0b729d571a44", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" + } + } + ], + "period": { + "start": "2019-11-08T20:05:11-08:00", + "end": "2019-11-08T20:20:11-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24", + "resource": { + "resourceType": "Condition", + "id": "3e0d8841-6fc1-477f-9912-6db4d8720d24", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44" + }, + "onsetDateTime": "2019-11-08T20:05:11-08:00", + "abatementDateTime": "2019-11-22T20:05:11-08:00", + "assertedDate": "2019-11-08T20:05:11-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:528c31eb-2246-4b58-a9a2-07fb59b381d9", + "resource": { + "resourceType": "Claim", + "id": "528c31eb-2246-4b58-a9a2-07fb59b381d9", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2019-11-08T20:05:11-08:00", + "end": "2019-11-08T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4c7d6ac2-ae54-4152-bff0-9a09cc05ff51", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4c7d6ac2-ae54-4152-bff0-9a09cc05ff51", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "528c31eb-2246-4b58-a9a2-07fb59b381d9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-11-08T20:20:11-08:00", + "end": "2020-11-08T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" + } + }, + "organization": { + "identifier": { + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f", + "resource": { + "resourceType": "Encounter", + "id": "145b1c95-c364-4a78-a844-f5a67043542f", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" + } + } + ], + "period": { + "start": "2020-01-01T20:05:11-08:00", + "end": "2020-01-01T20:20:11-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cf0c3103-f045-4c0f-b99e-c6be04e15933", + "resource": { + "resourceType": "Observation", + "id": "cf0c3103-f045-4c0f-b99e-c6be04e15933", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 123, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab2a369c-b257-4dd6-bd34-fd344763a07f", + "resource": { + "resourceType": "Observation", + "id": "ab2a369c-b257-4dd6-bd34-fd344763a07f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:97d3300a-8130-4fd5-be67-6edb249e749b", + "resource": { + "resourceType": "Observation", + "id": "97d3300a-8130-4fd5-be67-6edb249e749b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 24.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a0a9b1f3-201d-4f98-8b65-902c37885d5f", + "resource": { + "resourceType": "Observation", + "id": "a0a9b1f3-201d-4f98-8b65-902c37885d5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 15.970, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:363dee6e-a169-4914-a6bd-874ede37c24b", + "resource": { + "resourceType": "Observation", + "id": "363dee6e-a169-4914-a6bd-874ede37c24b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 55.212, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33fa8ad5-ae8f-4051-9a83-bdb1a39bdf9a", + "resource": { + "resourceType": "Observation", + "id": "33fa8ad5-ae8f-4051-9a83-bdb1a39bdf9a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 104, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab87697b-c351-4799-a1b9-e2448154dd2f", + "resource": { + "resourceType": "Observation", + "id": "ab87697b-c351-4799-a1b9-e2448154dd2f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 87, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9603d7c0-474c-4783-a3cd-2bb3e1173cc7", + "resource": { + "resourceType": "Observation", + "id": "9603d7c0-474c-4783-a3cd-2bb3e1173cc7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ba1da30b-a992-4066-ba97-7ccd956997df", + "resource": { + "resourceType": "Observation", + "id": "ba1da30b-a992-4066-ba97-7ccd956997df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "context": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "effectiveDateTime": "2020-01-01T20:05:11-08:00", + "issued": "2020-01-01T20:05:11.927-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7282f34b-e3a6-4037-81f0-e9d6eac7c6b3", + "resource": { + "resourceType": "Immunization", + "id": "7282f34b-e3a6-4037-81f0-e9d6eac7c6b3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "encounter": { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + }, + "date": "2020-01-01T20:05:11-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:337b8fa6-bdbd-4019-9aff-068fab01cdd2", + "resource": { + "resourceType": "Claim", + "id": "337b8fa6-bdbd-4019-9aff-068fab01cdd2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "start": "2020-01-01T20:05:11-08:00", + "end": "2020-01-01T20:20:11-08:00" + }, + "organization": { + "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7282f34b-e3a6-4037-81f0-e9d6eac7c6b3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b2f2ebb2-a126-470e-bb1a-58511e87f5ec", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b2f2ebb2-a126-470e-bb1a-58511e87f5ec", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "337b8fa6-bdbd-4019-9aff-068fab01cdd2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2020-01-01T20:20:11-08:00", + "end": "2021-01-01T20:20:11-08:00" + }, + "provider": { + "identifier": { + "value": "ba314750-701e-3370-ade0-270e178d80e0" + } + }, + "organization": { + "identifier": { + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json new file mode 100644 index 000000000000..47d9b906b987 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json @@ -0,0 +1,18231 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed", + "resource": { + "resourceType": "Patient", + "id": "74779846-85a4-4b26-9da4-414a5fec1aed", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 1459817271572172624 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2106-3", + "display": "White" + } + }, + { + "url": "text", + "valueString": "White" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Verlie371 Reynolds644" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Easthampton", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.012217100297429643 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 10.98778289970257 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "74779846-85a4-4b26-9da4-414a5fec1aed" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "74779846-85a4-4b26-9da4-414a5fec1aed" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-42-4253" + } + ], + "name": [ + { + "use": "official", + "family": "Maggio310", + "given": [ + "Dannette613" + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-178-3601", + "use": "home" + } + ], + "gender": "female", + "birthDate": "2008-05-01", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.12613043432252 + }, + { + "url": "longitude", + "valueDecimal": -70.96006577849847 + } + ] + } + ], + "line": [ + "291 Thiel Spur" + ], + "city": "Whitman", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthBoolean": false, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", + "resource": { + "resourceType": "Organization", + "id": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SIGNATURE HEALTHCARE MEDICAL GROUP INC", + "telecom": [ + { + "system": "phone", + "value": "781-878-1700" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.119966999999995 + }, + { + "url": "longitude", + "valueDecimal": -70.957211 + } + ] + } + ], + "line": [ + "360 BROCKTON AVE" + ], + "city": "ABINGTON", + "state": "MA", + "postalCode": "02351-2186", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841", + "resource": { + "resourceType": "Practitioner", + "id": "3dde31b6-dbc2-394b-b2af-a3167d1f6841", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "81770" + } + ], + "active": true, + "name": [ + { + "family": "Ondricka197", + "given": [ + "Leisa54" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "360 BROCKTON AVE" + ], + "city": "ABINGTON", + "state": "MA", + "postalCode": "02351-2186", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927", + "resource": { + "resourceType": "Encounter", + "id": "e5ebcd42-3144-4f74-b410-0e924137a927", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2010-10-07T23:41:28-07:00", + "end": "2010-10-07T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:34a8192f-b7a2-4ffc-b836-a9e44941009d", + "resource": { + "resourceType": "Observation", + "id": "34a8192f-b7a2-4ffc-b836-a9e44941009d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 92.600, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a36d03b9-1920-424e-bd4a-fb5bebdda8a7", + "resource": { + "resourceType": "Observation", + "id": "a36d03b9-1920-424e-bd4a-fb5bebdda8a7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eff3d32b-907b-48be-b94d-6c26f1ea361e", + "resource": { + "resourceType": "Observation", + "id": "eff3d32b-907b-48be-b94d-6c26f1ea361e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 16.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:300c5bc5-ff4f-4bfc-b6c8-57cba5943212", + "resource": { + "resourceType": "Observation", + "id": "300c5bc5-ff4f-4bfc-b6c8-57cba5943212", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 97.537, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68f2b87e-7d26-4614-92c7-1654538bc329", + "resource": { + "resourceType": "Observation", + "id": "68f2b87e-7d26-4614-92c7-1654538bc329", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 49.260, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:332ca9b2-e15e-437c-86a7-d154b14bfda5", + "resource": { + "resourceType": "Observation", + "id": "332ca9b2-e15e-437c-86a7-d154b14bfda5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 18.990, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:491d72ab-c2b5-4971-b96c-316507e3e131", + "resource": { + "resourceType": "Observation", + "id": "491d72ab-c2b5-4971-b96c-316507e3e131", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.594, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c27dfbc6-bb82-436d-9b76-99da730928bd", + "resource": { + "resourceType": "Observation", + "id": "c27dfbc6-bb82-436d-9b76-99da730928bd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 70, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 135, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32c8cddb-fdc8-4e80-9415-557e16059689", + "resource": { + "resourceType": "Observation", + "id": "32c8cddb-fdc8-4e80-9415-557e16059689", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 83, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:74c5f339-8b2a-4ed9-92ea-3f3c76ac743f", + "resource": { + "resourceType": "Observation", + "id": "74c5f339-8b2a-4ed9-92ea-3f3c76ac743f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4cff6afb-2c40-4844-9f44-68f758947cea", + "resource": { + "resourceType": "Observation", + "id": "4cff6afb-2c40-4844-9f44-68f758947cea", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "effectiveDateTime": "2010-10-07T23:41:28-07:00", + "issued": "2010-10-07T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8fe227f5-c731-47bf-b581-f9f060994298", + "resource": { + "resourceType": "Immunization", + "id": "8fe227f5-c731-47bf-b581-f9f060994298", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "date": "2010-10-07T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:7c493b39-652c-43fd-b8f8-7bc2b13abd07", + "resource": { + "resourceType": "Immunization", + "id": "7c493b39-652c-43fd-b8f8-7bc2b13abd07", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "83", + "display": "Hep A, ped/adol, 2 dose" + } + ], + "text": "Hep A, ped/adol, 2 dose" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + }, + "date": "2010-10-07T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:01cc5efc-a50e-4e06-9c66-48479aa8ea5a", + "resource": { + "resourceType": "Claim", + "id": "01cc5efc-a50e-4e06-9c66-48479aa8ea5a", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2010-10-07T23:41:28-07:00", + "end": "2010-10-07T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8fe227f5-c731-47bf-b581-f9f060994298" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:7c493b39-652c-43fd-b8f8-7bc2b13abd07" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1c6b395b-df6d-484e-9531-6374352ca78c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1c6b395b-df6d-484e-9531-6374352ca78c", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "01cc5efc-a50e-4e06-9c66-48479aa8ea5a" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2010-10-07T23:56:28-07:00", + "end": "2011-10-07T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 224.83200000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", + "resource": { + "resourceType": "Organization", + "id": "c44f361c-2efb-3050-8f97-0354a12e2920", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "c44f361c-2efb-3050-8f97-0354a12e2920" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5089417000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", + "resource": { + "resourceType": "Practitioner", + "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "200" + } + ], + "active": true, + "name": [ + { + "family": "Padberg411", + "given": [ + "Nila48" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737", + "resource": { + "resourceType": "Encounter", + "id": "3309eef4-cf71-4b37-9270-1e5914307737", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" + } + } + ], + "period": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf", + "resource": { + "resourceType": "Condition", + "id": "72b92bb7-8725-46fc-a14c-611dda67bdcf", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "65363002", + "display": "Otitis media" + } + ], + "text": "Otitis media" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + }, + "onsetDateTime": "2010-12-17T22:41:28-08:00", + "abatementDateTime": "2011-04-07T23:41:28-07:00", + "assertedDate": "2010-12-17T22:41:28-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:58c42292-b70d-4472-929d-699e52f1c77c", + "resource": { + "resourceType": "MedicationRequest", + "id": "58c42292-b70d-4472-929d-699e52f1c77c", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "309097", + "display": "Cefuroxime 250 MG Oral Tablet" + } + ], + "text": "Cefuroxime 250 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + }, + "authoredOn": "2010-12-17T22:41:28-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" + }, + "onBehalfOf": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0705a7f2-dafa-4ad3-b326-b0c3962ecc25", + "resource": { + "resourceType": "Claim", + "id": "0705a7f2-dafa-4ad3-b326-b0c3962ecc25", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + }, + "prescription": { + "reference": "urn:uuid:58c42292-b70d-4472-929d-699e52f1c77c" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + } + ] + } + ], + "total": { + "value": 50.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4db09a9f-a5fa-41a1-a0eb-d35403e1d53d", + "resource": { + "resourceType": "MedicationRequest", + "id": "4db09a9f-a5fa-41a1-a0eb-d35403e1d53d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + }, + "authoredOn": "2010-12-17T22:41:28-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" + }, + "onBehalfOf": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:9cb8f511-3e89-43e1-8874-9193b5873dd4", + "resource": { + "resourceType": "Claim", + "id": "9cb8f511-3e89-43e1-8874-9193b5873dd4", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + }, + "prescription": { + "reference": "urn:uuid:4db09a9f-a5fa-41a1-a0eb-d35403e1d53d" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + } + ] + } + ], + "total": { + "value": 7.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:20b8fb5e-4f8c-49b9-8abb-952ee2087bb6", + "resource": { + "resourceType": "Claim", + "id": "20b8fb5e-4f8c-49b9-8abb-952ee2087bb6", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b721dfed-3274-4146-8667-5483ee669030", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b721dfed-3274-4146-8667-5483ee669030", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "NO_INSURANCE" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "20b8fb5e-4f8c-49b9-8abb-952ee2087bb6" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2010-12-17T22:56:28-08:00", + "end": "2011-12-17T22:56:28-08:00" + }, + "created": "2010-12-17T22:56:28-08:00", + "provider": { + "identifier": { + "value": "d08d5908-40a4-3571-8c54-0cc029c8277f" + } + }, + "organization": { + "identifier": { + "value": "c44f361c-2efb-3050-8f97-0354a12e2920" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:20b8fb5e-4f8c-49b9-8abb-952ee2087bb6" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2010-12-17T22:41:28-08:00", + "end": "2010-12-17T22:56:28-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", + "resource": { + "resourceType": "Organization", + "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "GOOD SAMARITAN MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5084273000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "resource": { + "resourceType": "Practitioner", + "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "460" + } + ], + "active": true, + "name": [ + { + "family": "Fay398", + "given": [ + "Deneen201" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e", + "resource": { + "resourceType": "Encounter", + "id": "f3312988-276e-4486-83d8-d1d00f8fb52e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2011-02-21T22:41:28-08:00", + "end": "2011-02-21T22:56:28-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4", + "resource": { + "resourceType": "Condition", + "id": "f482fe43-8829-4943-ae76-59a8e03be4a4", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e" + }, + "onsetDateTime": "2011-02-21T22:41:28-08:00", + "abatementDateTime": "2011-02-28T22:41:28-08:00", + "assertedDate": "2011-02-21T22:41:28-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:9d51479d-a632-4b0c-a35c-ab75b010ef5c", + "resource": { + "resourceType": "Claim", + "id": "9d51479d-a632-4b0c-a35c-ab75b010ef5c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2011-02-21T22:41:28-08:00", + "end": "2011-02-21T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4124dba5-52f0-4064-aa61-ad5c311515c1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4124dba5-52f0-4064-aa61-ad5c311515c1", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9d51479d-a632-4b0c-a35c-ab75b010ef5c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2011-02-21T22:56:28-08:00", + "end": "2012-02-21T22:56:28-08:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88", + "resource": { + "resourceType": "Encounter", + "id": "9d0abe12-fe52-440b-be7c-a4f901b48e88", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2011-04-07T23:41:28-07:00", + "end": "2011-04-07T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d6dc6847-3da1-4a29-95ce-f43cfdf02dbb", + "resource": { + "resourceType": "Observation", + "id": "d6dc6847-3da1-4a29-95ce-f43cfdf02dbb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.800, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0ae4be2c-e803-4f78-992f-f19340441ca6", + "resource": { + "resourceType": "Observation", + "id": "0ae4be2c-e803-4f78-992f-f19340441ca6", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7a7e71ee-d58d-445e-8590-69f4b1f53552", + "resource": { + "resourceType": "Observation", + "id": "7a7e71ee-d58d-445e-8590-69f4b1f53552", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 17.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e85dca4a-2cbf-4e48-a7a3-6096fc9dc0c8", + "resource": { + "resourceType": "Observation", + "id": "e85dca4a-2cbf-4e48-a7a3-6096fc9dc0c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.701, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c4a8a86a-687f-43a2-9e3b-171d01a728dc", + "resource": { + "resourceType": "Observation", + "id": "c4a8a86a-687f-43a2-9e3b-171d01a728dc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 49.820, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed8a2cce-c94c-48cc-89dd-524b8cc46a5f", + "resource": { + "resourceType": "Observation", + "id": "ed8a2cce-c94c-48cc-89dd-524b8cc46a5f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 18.700, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6c442a5-657f-4939-81b9-db8096209d8f", + "resource": { + "resourceType": "Observation", + "id": "e6c442a5-657f-4939-81b9-db8096209d8f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.728, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:52b5eb5c-3f7d-439b-9563-48b50e200baa", + "resource": { + "resourceType": "Observation", + "id": "52b5eb5c-3f7d-439b-9563-48b50e200baa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 115, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1e36554-c799-40d2-a3e5-89d72a353ad8", + "resource": { + "resourceType": "Observation", + "id": "d1e36554-c799-40d2-a3e5-89d72a353ad8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 78, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c458ab22-4b31-4d4b-a688-a718c567c638", + "resource": { + "resourceType": "Observation", + "id": "c458ab22-4b31-4d4b-a688-a718c567c638", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1e9f2237-a788-4911-8683-8a34a715e498", + "resource": { + "resourceType": "Observation", + "id": "1e9f2237-a788-4911-8683-8a34a715e498", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "effectiveDateTime": "2011-04-07T23:41:28-07:00", + "issued": "2011-04-07T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dfb4acb6-d6aa-4d4e-9797-e594ed46ea34", + "resource": { + "resourceType": "Immunization", + "id": "dfb4acb6-d6aa-4d4e-9797-e594ed46ea34", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "date": "2011-04-07T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:58fa92cc-4318-4e1b-a2df-5d79549b1e4b", + "resource": { + "resourceType": "Immunization", + "id": "58fa92cc-4318-4e1b-a2df-5d79549b1e4b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + }, + "date": "2011-04-07T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:17d7579d-29f1-41a6-b57d-5449b24318f5", + "resource": { + "resourceType": "Claim", + "id": "17d7579d-29f1-41a6-b57d-5449b24318f5", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2011-04-07T23:41:28-07:00", + "end": "2011-04-07T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:dfb4acb6-d6aa-4d4e-9797-e594ed46ea34" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:58fa92cc-4318-4e1b-a2df-5d79549b1e4b" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:717c1332-5233-4f94-8200-6a105acd9bca", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "717c1332-5233-4f94-8200-6a105acd9bca", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "17d7579d-29f1-41a6-b57d-5449b24318f5" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2011-04-07T23:56:28-07:00", + "end": "2012-04-07T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 224.83200000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4", + "resource": { + "resourceType": "Encounter", + "id": "2db4ee31-f666-4d2e-bc74-7fabd9ca59d4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2011-10-06T23:41:28-07:00", + "end": "2011-10-07T00:11:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:f09a50e7-cebd-45b5-bda1-cb5e34a46b1a", + "resource": { + "resourceType": "Observation", + "id": "f09a50e7-cebd-45b5-bda1-cb5e34a46b1a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 100.40, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:727f4b88-7381-41fa-a6d8-663112a6d8d1", + "resource": { + "resourceType": "Observation", + "id": "727f4b88-7381-41fa-a6d8-663112a6d8d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:735fc3c8-0e13-4832-a490-c703c645067a", + "resource": { + "resourceType": "Observation", + "id": "735fc3c8-0e13-4832-a490-c703c645067a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 18.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5d229eb-6cf7-412d-b68b-2afe39d83ad7", + "resource": { + "resourceType": "Observation", + "id": "a5d229eb-6cf7-412d-b68b-2afe39d83ad7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 97.324, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d9f635cb-b6a6-4c32-be65-371d7cde5186", + "resource": { + "resourceType": "Observation", + "id": "d9f635cb-b6a6-4c32-be65-371d7cde5186", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 49.870, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:939f3fe0-9125-4ffd-8d5a-d86dfc136ad7", + "resource": { + "resourceType": "Observation", + "id": "939f3fe0-9125-4ffd-8d5a-d86dfc136ad7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 18.160, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:16724756-bd97-4aef-945f-e1b4719d6d42", + "resource": { + "resourceType": "Observation", + "id": "16724756-bd97-4aef-945f-e1b4719d6d42", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 95.278, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9d09c1be-c425-4f5e-b86e-7584d9af2f4e", + "resource": { + "resourceType": "Observation", + "id": "9d09c1be-c425-4f5e-b86e-7584d9af2f4e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 109, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d77fc935-f60a-44d4-9224-5c3b097298fa", + "resource": { + "resourceType": "Observation", + "id": "d77fc935-f60a-44d4-9224-5c3b097298fa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 93, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f09965b0-154e-4160-b64d-045792c8b125", + "resource": { + "resourceType": "Observation", + "id": "f09965b0-154e-4160-b64d-045792c8b125", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:77fd6cae-569e-40d9-865a-e79094dda946", + "resource": { + "resourceType": "Observation", + "id": "77fd6cae-569e-40d9-865a-e79094dda946", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "effectiveDateTime": "2011-10-06T23:41:28-07:00", + "issued": "2011-10-06T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd2964ac-95bb-4828-9e15-94e45efaf6a6", + "resource": { + "resourceType": "Procedure", + "id": "cd2964ac-95bb-4828-9e15-94e45efaf6a6", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "performedPeriod": { + "start": "2011-10-06T23:41:28-07:00", + "end": "2011-10-06T23:56:28-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4088ae81-89e9-44c9-8122-cc344132f3cf", + "resource": { + "resourceType": "Immunization", + "id": "4088ae81-89e9-44c9-8122-cc344132f3cf", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "133", + "display": "Pneumococcal conjugate PCV 13" + } + ], + "text": "Pneumococcal conjugate PCV 13" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + }, + "date": "2011-10-06T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e2bb83a3-ad13-461b-9ec5-bbd4f475f736", + "resource": { + "resourceType": "Claim", + "id": "e2bb83a3-ad13-461b-9ec5-bbd4f475f736", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2011-10-06T23:41:28-07:00", + "end": "2011-10-07T00:11:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4088ae81-89e9-44c9-8122-cc344132f3cf" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:cd2964ac-95bb-4828-9e15-94e45efaf6a6" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 670.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:43f21ab4-7c6a-4854-8c1f-ae6809c142ab", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "43f21ab4-7c6a-4854-8c1f-ae6809c142ab", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e2bb83a3-ad13-461b-9ec5-bbd4f475f736" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2011-10-07T00:11:28-07:00", + "end": "2012-10-07T00:11:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 134.052, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 536.208, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 670.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 670.26, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 648.624, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1", + "resource": { + "resourceType": "Encounter", + "id": "183e5799-9baa-4b23-bb26-e258f79c7bc1", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2011-11-02T23:41:28-07:00", + "end": "2011-11-02T23:56:28-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9", + "resource": { + "resourceType": "Condition", + "id": "7d48f9ed-889b-4cf3-aafd-69a9c4a587b9", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "444814009", + "display": "Viral sinusitis (disorder)" + } + ], + "text": "Viral sinusitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1" + }, + "onsetDateTime": "2011-11-02T23:41:28-07:00", + "abatementDateTime": "2011-11-23T22:41:28-08:00", + "assertedDate": "2011-11-02T23:41:28-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:bcdf9318-633d-4391-a3d8-ec701283f38c", + "resource": { + "resourceType": "Claim", + "id": "bcdf9318-633d-4391-a3d8-ec701283f38c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2011-11-02T23:41:28-07:00", + "end": "2011-11-02T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4b900b30-5799-4f1d-9b04-7853a9e1550d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4b900b30-5799-4f1d-9b04-7853a9e1550d", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bcdf9318-633d-4391-a3d8-ec701283f38c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2011-11-02T23:56:28-07:00", + "end": "2012-11-02T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d", + "resource": { + "resourceType": "Encounter", + "id": "22a11f12-80f8-41bc-98eb-f68059caa86d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2012-03-11T23:41:28-07:00", + "end": "2012-03-12T00:41:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa", + "resource": { + "resourceType": "Condition", + "id": "3697931b-452d-4681-aa9a-1cf191ad96aa", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "62106007", + "display": "Concussion with no loss of consciousness" + } + ], + "text": "Concussion with no loss of consciousness" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" + }, + "onsetDateTime": "2012-03-11T23:41:28-07:00", + "abatementDateTime": "2012-05-10T23:41:28-07:00", + "assertedDate": "2012-03-11T23:41:28-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ca139494-e8a0-4147-bcfb-f1f05237c1ce", + "resource": { + "resourceType": "CarePlan", + "id": "ca139494-e8a0-4147-bcfb-f1f05237c1ce", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "47387005", + "display": "Head injury rehabilitation" + } + ], + "text": "Head injury rehabilitation" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" + }, + "period": { + "start": "2012-03-11T23:41:28-07:00", + "end": "2012-05-10T23:41:28-07:00" + }, + "addresses": [ + { + "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "183051005", + "display": "Recommendation to rest" + } + ], + "text": "Recommendation to rest" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "226138001", + "display": "Alcohol-free diet" + } + ], + "text": "Alcohol-free diet" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:d075f4b5-3bdf-4e3c-8d39-97f665898dec", + "resource": { + "resourceType": "Claim", + "id": "d075f4b5-3bdf-4e3c-8d39-97f665898dec", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2012-03-11T23:41:28-07:00", + "end": "2012-03-12T00:41:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:041ee73f-a2b7-485e-9c9a-a3ce61d61aeb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "041ee73f-a2b7-485e-9c9a-a3ce61d61aeb", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d075f4b5-3bdf-4e3c-8d39-97f665898dec" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2012-03-12T00:41:28-07:00", + "end": "2013-03-12T00:41:28-07:00" + }, + "created": "2012-03-12T00:41:28-07:00", + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:d075f4b5-3bdf-4e3c-8d39-97f665898dec" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2012-03-11T23:41:28-07:00", + "end": "2012-03-12T00:41:28-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2012-03-11T23:41:28-07:00", + "end": "2012-03-12T00:41:28-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c", + "resource": { + "resourceType": "Encounter", + "id": "2f6a8179-14c6-48c5-8e16-83b0c28c8e3c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2012-04-05T23:41:28-07:00", + "end": "2012-04-05T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:530fb772-2ae3-49c3-9584-8ece6fb95b78", + "resource": { + "resourceType": "Observation", + "id": "530fb772-2ae3-49c3-9584-8ece6fb95b78", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 104, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e9b47b26-3902-4d1c-b6bb-e7fb1f100450", + "resource": { + "resourceType": "Observation", + "id": "e9b47b26-3902-4d1c-b6bb-e7fb1f100450", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee398553-755c-4270-9e0a-6cbc2fde81b9", + "resource": { + "resourceType": "Observation", + "id": "ee398553-755c-4270-9e0a-6cbc2fde81b9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 19.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:584a6540-94e5-4631-b228-ecaa0c3940ee", + "resource": { + "resourceType": "Observation", + "id": "584a6540-94e5-4631-b228-ecaa0c3940ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "77606-2", + "display": "Weight-for-length Per age and sex" + } + ], + "text": "Weight-for-length Per age and sex" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 97.324, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8a70ed08-c562-4c20-b0a0-b9975c95a37c", + "resource": { + "resourceType": "Observation", + "id": "8a70ed08-c562-4c20-b0a0-b9975c95a37c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9843-4", + "display": "Head Occipital-frontal circumference" + } + ], + "text": "Head Occipital-frontal circumference" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 49.870, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3bb64db9-2e24-46fe-99ae-2be4476569b7", + "resource": { + "resourceType": "Observation", + "id": "3bb64db9-2e24-46fe-99ae-2be4476569b7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 17.790, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7195cbc2-53fe-4d0f-bf39-03573a548216", + "resource": { + "resourceType": "Observation", + "id": "7195cbc2-53fe-4d0f-bf39-03573a548216", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 93.806, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:860a47c4-6065-44bb-936e-c23cfcd14d91", + "resource": { + "resourceType": "Observation", + "id": "860a47c4-6065-44bb-936e-c23cfcd14d91", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:992ec7de-2724-4a42-931c-d844c5aafe12", + "resource": { + "resourceType": "Observation", + "id": "992ec7de-2724-4a42-931c-d844c5aafe12", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 69, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4eba63ad-1102-4a8e-860d-9625060c484d", + "resource": { + "resourceType": "Observation", + "id": "4eba63ad-1102-4a8e-860d-9625060c484d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed62ee67-1921-408e-ae5f-8210589c2530", + "resource": { + "resourceType": "Observation", + "id": "ed62ee67-1921-408e-ae5f-8210589c2530", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "effectiveDateTime": "2012-04-05T23:41:28-07:00", + "issued": "2012-04-05T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:63607712-e6ad-4ad0-8374-167292d2b732", + "resource": { + "resourceType": "Immunization", + "id": "63607712-e6ad-4ad0-8374-167292d2b732", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + }, + "date": "2012-04-05T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:44ff0e2e-37ea-46cc-90ca-7dec78716119", + "resource": { + "resourceType": "Claim", + "id": "44ff0e2e-37ea-46cc-90ca-7dec78716119", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2012-04-05T23:41:28-07:00", + "end": "2012-04-05T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:63607712-e6ad-4ad0-8374-167292d2b732" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:01acdc68-c657-4980-b249-f74d2bde54e3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "01acdc68-c657-4980-b249-f74d2bde54e3", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "44ff0e2e-37ea-46cc-90ca-7dec78716119" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-04-05T23:56:28-07:00", + "end": "2013-04-05T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4a42aa41-c430-4bf7-8a27-b29bbe6d7d50", + "resource": { + "resourceType": "Encounter", + "id": "4a42aa41-c430-4bf7-8a27-b29bbe6d7d50", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2012-05-10T23:41:28-07:00", + "end": "2012-05-10T23:56:28-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "62106007", + "display": "Concussion with no loss of consciousness" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1b21ca69-4933-47fc-9362-d0506cf3d2ab", + "resource": { + "resourceType": "Claim", + "id": "1b21ca69-4933-47fc-9362-d0506cf3d2ab", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2012-05-10T23:41:28-07:00", + "end": "2012-05-10T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4a42aa41-c430-4bf7-8a27-b29bbe6d7d50" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:2ff6e675-a27b-48dc-9ed5-88c4202d0d34", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "2ff6e675-a27b-48dc-9ed5-88c4202d0d34", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1b21ca69-4933-47fc-9362-d0506cf3d2ab" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-05-10T23:56:28-07:00", + "end": "2013-05-10T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69", + "resource": { + "resourceType": "Encounter", + "id": "45c0b635-4235-491c-b1ff-9ffb7c7f8e69", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2013-04-11T23:41:28-07:00", + "end": "2013-04-11T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2b2973ea-c054-44fa-9572-e77b63e25874", + "resource": { + "resourceType": "Observation", + "id": "2b2973ea-c054-44fa-9572-e77b63e25874", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 111.30, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:917b89dc-e9ac-4faa-a652-17fbd36febb2", + "resource": { + "resourceType": "Observation", + "id": "917b89dc-e9ac-4faa-a652-17fbd36febb2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:52932aa8-82f9-4594-aefb-f6c68962dc58", + "resource": { + "resourceType": "Observation", + "id": "52932aa8-82f9-4594-aefb-f6c68962dc58", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 24.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7bc13f9d-54d8-40d2-b250-70d601baaec9", + "resource": { + "resourceType": "Observation", + "id": "7bc13f9d-54d8-40d2-b250-70d601baaec9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 19.770, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ebaeae02-06a2-4cbf-84c3-a6fedee5518c", + "resource": { + "resourceType": "Observation", + "id": "ebaeae02-06a2-4cbf-84c3-a6fedee5518c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 98.235, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ef996b79-7daf-4fe6-8657-dfc8d3b8825e", + "resource": { + "resourceType": "Observation", + "id": "ef996b79-7daf-4fe6-8657-dfc8d3b8825e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 118, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d530ecb0-4846-47e1-8fec-a8862790479a", + "resource": { + "resourceType": "Observation", + "id": "d530ecb0-4846-47e1-8fec-a8862790479a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 71, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee6c080f-aec2-4c79-b486-fb59a4a8a435", + "resource": { + "resourceType": "Observation", + "id": "ee6c080f-aec2-4c79-b486-fb59a4a8a435", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:80680d3c-a823-49d9-829b-f136f7bd140c", + "resource": { + "resourceType": "Observation", + "id": "80680d3c-a823-49d9-829b-f136f7bd140c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "effectiveDateTime": "2013-04-11T23:41:28-07:00", + "issued": "2013-04-11T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f198a0ac-e871-49f4-a716-67c260ce83b2", + "resource": { + "resourceType": "Immunization", + "id": "f198a0ac-e871-49f4-a716-67c260ce83b2", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "21", + "display": "varicella" + } + ], + "text": "varicella" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "date": "2013-04-11T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4155a7be-e200-4fd4-a5f0-a9d2dcfb977c", + "resource": { + "resourceType": "Immunization", + "id": "4155a7be-e200-4fd4-a5f0-a9d2dcfb977c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "10", + "display": "IPV" + } + ], + "text": "IPV" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "date": "2013-04-11T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543", + "resource": { + "resourceType": "Immunization", + "id": "8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "date": "2013-04-11T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:edee72cd-efe2-4fb4-abb6-dd19a1941511", + "resource": { + "resourceType": "Immunization", + "id": "edee72cd-efe2-4fb4-abb6-dd19a1941511", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "20", + "display": "DTaP" + } + ], + "text": "DTaP" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "date": "2013-04-11T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c4532309-bca7-4039-bca0-646214dd4157", + "resource": { + "resourceType": "Immunization", + "id": "c4532309-bca7-4039-bca0-646214dd4157", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "03", + "display": "MMR" + } + ], + "text": "MMR" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + }, + "date": "2013-04-11T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:2d47359e-0d62-46d6-8479-6bac149d5a13", + "resource": { + "resourceType": "Claim", + "id": "2d47359e-0d62-46d6-8479-6bac149d5a13", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2013-04-11T23:41:28-07:00", + "end": "2013-04-11T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f198a0ac-e871-49f4-a716-67c260ce83b2" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:4155a7be-e200-4fd4-a5f0-a9d2dcfb977c" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:edee72cd-efe2-4fb4-abb6-dd19a1941511" + } + }, + { + "sequence": 5, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c4532309-bca7-4039-bca0-646214dd4157" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "informationLinkId": [ + 5 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5681015a-7192-4b17-a375-8ec036828a5e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5681015a-7192-4b17-a375-8ec036828a5e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2d47359e-0d62-46d6-8479-6bac149d5a13" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-04-11T23:56:28-07:00", + "end": "2014-04-11T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 562.08, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833", + "resource": { + "resourceType": "Encounter", + "id": "c19ef61d-add1-4fc9-808d-66c1dbe56833", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2014-04-17T23:41:28-07:00", + "end": "2014-04-17T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:2fa0933f-b5f9-4690-bc3a-940f6eec5020", + "resource": { + "resourceType": "Observation", + "id": "2fa0933f-b5f9-4690-bc3a-940f6eec5020", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 118.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8e3f3cad-428a-47e6-89e9-b136d6c32682", + "resource": { + "resourceType": "Observation", + "id": "8e3f3cad-428a-47e6-89e9-b136d6c32682", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dea4beea-7c95-470f-828d-0fc6b675beb4", + "resource": { + "resourceType": "Observation", + "id": "dea4beea-7c95-470f-828d-0fc6b675beb4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 27.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:db434f82-d9cf-4d8a-8db9-98214c182999", + "resource": { + "resourceType": "Observation", + "id": "db434f82-d9cf-4d8a-8db9-98214c182999", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 19.310, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:030cfeb0-62a9-4bd6-9385-5dac07a65a7f", + "resource": { + "resourceType": "Observation", + "id": "030cfeb0-62a9-4bd6-9385-5dac07a65a7f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.331, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:712c3100-3f61-4822-be80-fd18f9b3d10c", + "resource": { + "resourceType": "Observation", + "id": "712c3100-3f61-4822-be80-fd18f9b3d10c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 82, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1ee050f4-ee26-49bf-99fb-043f1c72489c", + "resource": { + "resourceType": "Observation", + "id": "1ee050f4-ee26-49bf-99fb-043f1c72489c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 65, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8017673c-788f-40e0-9771-ccb93138fe94", + "resource": { + "resourceType": "Observation", + "id": "8017673c-788f-40e0-9771-ccb93138fe94", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a868b9da-3b19-4a35-8342-b4c60a646323", + "resource": { + "resourceType": "Observation", + "id": "a868b9da-3b19-4a35-8342-b4c60a646323", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 9.7187, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", + "resource": { + "resourceType": "Observation", + "id": "af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 4.9436, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", + "resource": { + "resourceType": "Observation", + "id": "1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 15.663, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2d1da5b8-2b96-457d-9969-dde39bc851fe", + "resource": { + "resourceType": "Observation", + "id": "2d1da5b8-2b96-457d-9969-dde39bc851fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 40.869, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:048979bc-47b8-4357-a8d4-492328189f99", + "resource": { + "resourceType": "Observation", + "id": "048979bc-47b8-4357-a8d4-492328189f99", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 84.145, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", + "resource": { + "resourceType": "Observation", + "id": "b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 32.970, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1f03a4d1-8772-4091-a08d-400c085bd2b8", + "resource": { + "resourceType": "Observation", + "id": "1f03a4d1-8772-4091-a08d-400c085bd2b8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 34.981, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0af5dbbc-f03f-455d-a835-f58628306992", + "resource": { + "resourceType": "Observation", + "id": "0af5dbbc-f03f-455d-a835-f58628306992", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 44.016, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d437a5b7-45db-4c87-a645-255360935b43", + "resource": { + "resourceType": "Observation", + "id": "d437a5b7-45db-4c87-a645-255360935b43", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 156.02, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e29e77b5-235d-4cfb-a691-e9586c37fabd", + "resource": { + "resourceType": "Observation", + "id": "e29e77b5-235d-4cfb-a691-e9586c37fabd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 448.52, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e9cd132f-c636-4791-95a7-ed45ca72a048", + "resource": { + "resourceType": "Observation", + "id": "e9cd132f-c636-4791-95a7-ed45ca72a048", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueQuantity": { + "value": 10.855, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d815087-aeac-46f5-947d-d77492ee4982", + "resource": { + "resourceType": "Observation", + "id": "1d815087-aeac-46f5-947d-d77492ee4982", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82c07d2a-8d8d-49e0-ac46-e0aa65a715ef", + "resource": { + "resourceType": "Immunization", + "id": "82c07d2a-8d8d-49e0-ac46-e0aa65a715ef", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "date": "2014-04-17T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4082a6bf-e557-4084-95f9-63d1308c8cf8", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4082a6bf-e557-4084-95f9-63d1308c8cf8", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + }, + "effectiveDateTime": "2014-04-17T23:41:28-07:00", + "issued": "2014-04-17T23:41:28.175-07:00", + "result": [ + { + "reference": "urn:uuid:a868b9da-3b19-4a35-8342-b4c60a646323", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:2d1da5b8-2b96-457d-9969-dde39bc851fe", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:048979bc-47b8-4357-a8d4-492328189f99", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:1f03a4d1-8772-4091-a08d-400c085bd2b8", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:0af5dbbc-f03f-455d-a835-f58628306992", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:d437a5b7-45db-4c87-a645-255360935b43", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:e29e77b5-235d-4cfb-a691-e9586c37fabd", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:e9cd132f-c636-4791-95a7-ed45ca72a048", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:0538b525-e131-43da-8175-11faa1dded37", + "resource": { + "resourceType": "Claim", + "id": "0538b525-e131-43da-8175-11faa1dded37", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2014-04-17T23:41:28-07:00", + "end": "2014-04-17T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:82c07d2a-8d8d-49e0-ac46-e0aa65a715ef" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4e7d7291-ab0d-4552-9cf5-db775b5ae812", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4e7d7291-ab0d-4552-9cf5-db775b5ae812", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "0538b525-e131-43da-8175-11faa1dded37" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-04-17T23:56:28-07:00", + "end": "2015-04-17T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527", + "resource": { + "resourceType": "Encounter", + "id": "fa0b0c4d-02fe-487e-9d31-20cdc2f7c527", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2015-03-28T23:41:28-07:00", + "end": "2015-03-29T00:11:28-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819", + "resource": { + "resourceType": "Condition", + "id": "461d127e-9029-4afd-b821-b35c230b6819", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" + }, + "onsetDateTime": "2015-03-28T23:41:28-07:00", + "abatementDateTime": "2015-04-06T23:41:28-07:00", + "assertedDate": "2015-03-28T23:41:28-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:095fb91f-d9a4-4b33-bd42-337d14b3488e", + "resource": { + "resourceType": "Observation", + "id": "095fb91f-d9a4-4b33-bd42-337d14b3488e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" + }, + "effectiveDateTime": "2015-03-28T23:41:28-07:00", + "issued": "2015-03-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 37.213, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5653153-cf11-41b0-b578-746ebdcff635", + "resource": { + "resourceType": "Procedure", + "id": "a5653153-cf11-41b0-b578-746ebdcff635", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "117015009", + "display": "Throat culture (procedure)" + } + ], + "text": "Throat culture (procedure)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" + }, + "performedPeriod": { + "start": "2015-03-28T23:41:28-07:00", + "end": "2015-03-28T23:56:28-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819", + "display": "Acute viral pharyngitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9d45adf5-e694-4b1c-9ad3-978e2e992ed7", + "resource": { + "resourceType": "Claim", + "id": "9d45adf5-e694-4b1c-9ad3-978e2e992ed7", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2015-03-28T23:41:28-07:00", + "end": "2015-03-29T00:11:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a5653153-cf11-41b0-b578-746ebdcff635" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "net": { + "value": 4036.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:87d56cb6-5891-4a0d-852c-c759ac33d286", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "87d56cb6-5891-4a0d-852c-c759ac33d286", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9d45adf5-e694-4b1c-9ad3-978e2e992ed7" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-03-29T00:11:28-07:00", + "end": "2016-03-29T00:11:28-07:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "117015009" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 807.244, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3228.976, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 4036.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 4036.22, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 3228.976, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5", + "resource": { + "resourceType": "Encounter", + "id": "cd225c5f-ea00-47da-aea5-f0672f13d3e5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2015-04-23T23:41:28-07:00", + "end": "2015-04-23T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5c275f33-5da3-4e8a-b8e1-7154a1ef6cf5", + "resource": { + "resourceType": "Observation", + "id": "5c275f33-5da3-4e8a-b8e1-7154a1ef6cf5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 125.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a3441f1-7561-46ce-8ede-8ee7d5df46c2", + "resource": { + "resourceType": "Observation", + "id": "1a3441f1-7561-46ce-8ede-8ee7d5df46c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:340d4f4f-afc4-4399-8efe-a070abf1232c", + "resource": { + "resourceType": "Observation", + "id": "340d4f4f-afc4-4399-8efe-a070abf1232c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 29.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:75a015c9-c028-4561-bcdd-aa168eb8bc60", + "resource": { + "resourceType": "Observation", + "id": "75a015c9-c028-4561-bcdd-aa168eb8bc60", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 18.710, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0e4f6eef-b361-4d4b-8b47-c862c2023862", + "resource": { + "resourceType": "Observation", + "id": "0e4f6eef-b361-4d4b-8b47-c862c2023862", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 92.067, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e8bdb47c-54d0-44b9-863a-76a19cb07442", + "resource": { + "resourceType": "Observation", + "id": "e8bdb47c-54d0-44b9-863a-76a19cb07442", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 77, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 133, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8bda595-7fc7-4f89-b4d4-384ad4928fb2", + "resource": { + "resourceType": "Observation", + "id": "b8bda595-7fc7-4f89-b4d4-384ad4928fb2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 71, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a60af7cd-62cf-4563-84bf-36b3506bfb17", + "resource": { + "resourceType": "Observation", + "id": "a60af7cd-62cf-4563-84bf-36b3506bfb17", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueQuantity": { + "value": 12, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f0dcf832-bd29-49ac-b4f6-1832349653e9", + "resource": { + "resourceType": "Observation", + "id": "f0dcf832-bd29-49ac-b4f6-1832349653e9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "effectiveDateTime": "2015-04-23T23:41:28-07:00", + "issued": "2015-04-23T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:15451914-2ac0-445d-9bea-55c407260c8f", + "resource": { + "resourceType": "Immunization", + "id": "15451914-2ac0-445d-9bea-55c407260c8f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + }, + "date": "2015-04-23T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:c410c39e-37e5-4884-828e-9e30fc01f547", + "resource": { + "resourceType": "Claim", + "id": "c410c39e-37e5-4884-828e-9e30fc01f547", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2015-04-23T23:41:28-07:00", + "end": "2015-04-23T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:15451914-2ac0-445d-9bea-55c407260c8f" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:bc77430b-e5cd-45c1-87eb-4664b60b49c9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "bc77430b-e5cd-45c1-87eb-4664b60b49c9", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c410c39e-37e5-4884-828e-9e30fc01f547" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-04-23T23:56:28-07:00", + "end": "2016-04-23T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de", + "resource": { + "resourceType": "Encounter", + "id": "1d9f5d50-0b15-423e-b7c8-7f76dad7a5de", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2016-04-28T23:41:28-07:00", + "end": "2016-04-29T00:11:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:beb1f62e-15cf-472a-8900-71908c69f007", + "resource": { + "resourceType": "Observation", + "id": "beb1f62e-15cf-472a-8900-71908c69f007", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 132.20, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:031335b4-b74a-4313-a3a6-6ec46092f24d", + "resource": { + "resourceType": "Observation", + "id": "031335b4-b74a-4313-a3a6-6ec46092f24d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9626ab5e-78a6-4a10-9a67-019efb780f1a", + "resource": { + "resourceType": "Observation", + "id": "9626ab5e-78a6-4a10-9a67-019efb780f1a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 41.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:47232ee4-8643-4c87-970a-5bc5993a6fc4", + "resource": { + "resourceType": "Observation", + "id": "47232ee4-8643-4c87-970a-5bc5993a6fc4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 23.670, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ab9bdfe9-2aff-4e60-b5cd-dccb3ec27990", + "resource": { + "resourceType": "Observation", + "id": "ab9bdfe9-2aff-4e60-b5cd-dccb3ec27990", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 98.383, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:19129d9f-0b61-42a9-b7a9-f65558e77e65", + "resource": { + "resourceType": "Observation", + "id": "19129d9f-0b61-42a9-b7a9-f65558e77e65", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 127, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:40b2e56f-566f-469c-9e54-d95ab4a6bc92", + "resource": { + "resourceType": "Observation", + "id": "40b2e56f-566f-469c-9e54-d95ab4a6bc92", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 78, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:69ee5599-3732-4333-8945-59467ca787a0", + "resource": { + "resourceType": "Observation", + "id": "69ee5599-3732-4333-8945-59467ca787a0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a99da671-1992-483e-95c7-5528ead3e793", + "resource": { + "resourceType": "Observation", + "id": "a99da671-1992-483e-95c7-5528ead3e793", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "effectiveDateTime": "2016-04-28T23:41:28-07:00", + "issued": "2016-04-28T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:096db481-b816-412d-ad68-424a0ac417e6", + "resource": { + "resourceType": "Procedure", + "id": "096db481-b816-412d-ad68-424a0ac417e6", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "performedPeriod": { + "start": "2016-04-28T23:41:28-07:00", + "end": "2016-04-28T23:56:28-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:65c0a039-6927-4754-ad69-9efb59cb2ba3", + "resource": { + "resourceType": "Immunization", + "id": "65c0a039-6927-4754-ad69-9efb59cb2ba3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + }, + "date": "2016-04-28T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2", + "resource": { + "resourceType": "Claim", + "id": "57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2016-04-28T23:41:28-07:00", + "end": "2016-04-29T00:11:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:65c0a039-6927-4754-ad69-9efb59cb2ba3" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:096db481-b816-412d-ad68-424a0ac417e6" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 552.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e02bbf46-7a92-4921-8a2c-a9d6d3825093", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e02bbf46-7a92-4921-8a2c-a9d6d3825093", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-04-29T00:11:28-07:00", + "end": "2017-04-29T00:11:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 110.474, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 441.896, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 552.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 552.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 554.312, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65", + "resource": { + "resourceType": "Encounter", + "id": "01b37d9a-3c03-4380-9c0e-2161b13c9f65", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2016-11-30T22:41:28-08:00", + "end": "2016-11-30T22:56:28-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5", + "resource": { + "resourceType": "Condition", + "id": "0819f7ff-bcf6-4fd5-bef2-003de0b4cad5", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "43878008", + "display": "Streptococcal sore throat (disorder)" + } + ], + "text": "Streptococcal sore throat (disorder)" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" + }, + "onsetDateTime": "2016-11-30T22:41:28-08:00", + "abatementDateTime": "2016-12-11T22:41:28-08:00", + "assertedDate": "2016-11-30T22:41:28-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:ccb86374-8d06-4763-8d81-5f5f38964b16", + "resource": { + "resourceType": "Observation", + "id": "ccb86374-8d06-4763-8d81-5f5f38964b16", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" + }, + "effectiveDateTime": "2016-11-30T22:41:28-08:00", + "issued": "2016-11-30T22:41:28.175-08:00", + "valueQuantity": { + "value": 38.853, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4b86d0b2-5bcd-40f3-b375-72e8bd3b9836", + "resource": { + "resourceType": "MedicationRequest", + "id": "4b86d0b2-5bcd-40f3-b375-72e8bd3b9836", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "834061", + "display": "Penicillin V Potassium 250 MG Oral Tablet" + } + ], + "text": "Penicillin V Potassium 250 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" + }, + "authoredOn": "2016-11-30T22:41:28-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + }, + "onBehalfOf": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:25860c6e-63d3-44ad-a475-ec6ea05f6ae8", + "resource": { + "resourceType": "Claim", + "id": "25860c6e-63d3-44ad-a475-ec6ea05f6ae8", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2016-11-30T22:41:28-08:00", + "end": "2016-11-30T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "prescription": { + "reference": "urn:uuid:4b86d0b2-5bcd-40f3-b375-72e8bd3b9836" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" + } + ] + } + ], + "total": { + "value": 12.27, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6244ee17-361f-40fc-9de8-c2619da63404", + "resource": { + "resourceType": "Claim", + "id": "6244ee17-361f-40fc-9de8-c2619da63404", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2016-11-30T22:41:28-08:00", + "end": "2016-11-30T22:56:28-08:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5f22491b-f8ec-489a-997f-aff564a1e9a9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5f22491b-f8ec-489a-997f-aff564a1e9a9", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "6244ee17-361f-40fc-9de8-c2619da63404" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-11-30T22:56:28-08:00", + "end": "2017-11-30T22:56:28-08:00" + }, + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63", + "resource": { + "resourceType": "Encounter", + "id": "d791bca0-64d4-4169-8b1c-26f821f1fe63", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2017-05-04T23:41:28-07:00", + "end": "2017-05-04T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3f33d17c-0ea5-43b0-87e6-e2a269a9aa82", + "resource": { + "resourceType": "Observation", + "id": "3f33d17c-0ea5-43b0-87e6-e2a269a9aa82", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 138.30, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d9ee8452-e3c4-4197-9d4c-9924bacf089b", + "resource": { + "resourceType": "Observation", + "id": "d9ee8452-e3c4-4197-9d4c-9924bacf089b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:36bbe7db-206f-4b9b-930d-77ed432cbcca", + "resource": { + "resourceType": "Observation", + "id": "36bbe7db-206f-4b9b-930d-77ed432cbcca", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 46.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bd5525f4-a346-4675-9f56-db1337711563", + "resource": { + "resourceType": "Observation", + "id": "bd5525f4-a346-4675-9f56-db1337711563", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 24.120, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:befd064f-b53a-4239-bb73-6f8a797a0750", + "resource": { + "resourceType": "Observation", + "id": "befd064f-b53a-4239-bb73-6f8a797a0750", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 97.701, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6aee833a-c46c-49d5-8271-e664bcf1a778", + "resource": { + "resourceType": "Observation", + "id": "6aee833a-c46c-49d5-8271-e664bcf1a778", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 125, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ffc79ca3-8ccf-4dfe-bcf1-16c5884131ad", + "resource": { + "resourceType": "Observation", + "id": "ffc79ca3-8ccf-4dfe-bcf1-16c5884131ad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 76, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b2711b15-1375-41d5-bf47-5a1f6227122a", + "resource": { + "resourceType": "Observation", + "id": "b2711b15-1375-41d5-bf47-5a1f6227122a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:eee4571d-d19f-4fa9-b7ee-f2039023da34", + "resource": { + "resourceType": "Observation", + "id": "eee4571d-d19f-4fa9-b7ee-f2039023da34", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "effectiveDateTime": "2017-05-04T23:41:28-07:00", + "issued": "2017-05-04T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:511d15bf-031e-42ca-8f59-4534d2feb31f", + "resource": { + "resourceType": "Immunization", + "id": "511d15bf-031e-42ca-8f59-4534d2feb31f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + }, + "date": "2017-05-04T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b3a8886b-db9a-4761-ab68-629ce30e90eb", + "resource": { + "resourceType": "Claim", + "id": "b3a8886b-db9a-4761-ab68-629ce30e90eb", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2017-05-04T23:41:28-07:00", + "end": "2017-05-04T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:511d15bf-031e-42ca-8f59-4534d2feb31f" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:be175f10-ba2d-4ce8-bdc2-1bd62979b84f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "be175f10-ba2d-4ce8-bdc2-1bd62979b84f", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b3a8886b-db9a-4761-ab68-629ce30e90eb" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2017-05-04T23:56:28-07:00", + "end": "2018-05-04T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed", + "resource": { + "resourceType": "Encounter", + "id": "bd967ad1-5a15-4686-9618-fded1705f3ed", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2018-05-10T23:41:28-07:00", + "end": "2018-05-10T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:80946ae4-c740-4dd4-859f-63fceab53d5d", + "resource": { + "resourceType": "Observation", + "id": "80946ae4-c740-4dd4-859f-63fceab53d5d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 143.70, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1724ca7a-5f9c-4cb3-83f1-a1a561ba5d6e", + "resource": { + "resourceType": "Observation", + "id": "1724ca7a-5f9c-4cb3-83f1-a1a561ba5d6e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:83eff34f-9189-4c30-b328-090fcfe6bba3", + "resource": { + "resourceType": "Observation", + "id": "83eff34f-9189-4c30-b328-090fcfe6bba3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 49.200, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99b0abc7-ec89-4299-b861-73d79626fe37", + "resource": { + "resourceType": "Observation", + "id": "99b0abc7-ec89-4299-b861-73d79626fe37", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 23.830, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5387c90c-a7e0-4836-b7e6-bc8821ea907d", + "resource": { + "resourceType": "Observation", + "id": "5387c90c-a7e0-4836-b7e6-bc8821ea907d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 96.200, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f9b31e6f-a59c-4073-a13c-27af3b4b5215", + "resource": { + "resourceType": "Observation", + "id": "f9b31e6f-a59c-4073-a13c-27af3b4b5215", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 122, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3bfdc40d-1ab0-4f94-84f8-81a44e651471", + "resource": { + "resourceType": "Observation", + "id": "3bfdc40d-1ab0-4f94-84f8-81a44e651471", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 85, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:61acc741-5ea8-450e-9d10-207e44579856", + "resource": { + "resourceType": "Observation", + "id": "61acc741-5ea8-450e-9d10-207e44579856", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:42f62174-98bc-4624-8a12-f321654f15ff", + "resource": { + "resourceType": "Observation", + "id": "42f62174-98bc-4624-8a12-f321654f15ff", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "effectiveDateTime": "2018-05-10T23:41:28-07:00", + "issued": "2018-05-10T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32c5f73e-c88e-49b0-8cf9-14af8aee1ab3", + "resource": { + "resourceType": "Immunization", + "id": "32c5f73e-c88e-49b0-8cf9-14af8aee1ab3", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + }, + "date": "2018-05-10T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:1a6aa059-53e1-4cff-b8e8-5243b001997c", + "resource": { + "resourceType": "Claim", + "id": "1a6aa059-53e1-4cff-b8e8-5243b001997c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2018-05-10T23:41:28-07:00", + "end": "2018-05-10T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:32c5f73e-c88e-49b0-8cf9-14af8aee1ab3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c8122e8b-4a3f-45a9-8e01-e52123646895", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c8122e8b-4a3f-45a9-8e01-e52123646895", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "1a6aa059-53e1-4cff-b8e8-5243b001997c" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-05-10T23:56:28-07:00", + "end": "2019-05-10T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252", + "resource": { + "resourceType": "Encounter", + "id": "93415003-cc07-4544-8c69-34c3a9fad252", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + } + ], + "period": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890", + "resource": { + "resourceType": "Condition", + "id": "01e122c3-3db5-49ee-b256-9cbe928b9890", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "283371005", + "display": "Laceration of forearm" + } + ], + "text": "Laceration of forearm" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + }, + "onsetDateTime": "2018-06-08T23:41:28-07:00", + "abatementDateTime": "2018-06-22T23:41:28-07:00", + "assertedDate": "2018-06-08T23:41:28-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:5baa99a7-6e14-4553-9a0e-7d425bc4b6cf", + "resource": { + "resourceType": "Procedure", + "id": "5baa99a7-6e14-4553-9a0e-7d425bc4b6cf", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "288086009", + "display": "Suture open wound" + } + ], + "text": "Suture open wound" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + }, + "performedPeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-08T23:53:28-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890", + "display": "Laceration of forearm" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4573a9d7-d985-4d8c-815b-b8ce3fe66769", + "resource": { + "resourceType": "MedicationRequest", + "id": "4573a9d7-d985-4d8c-815b-b8ce3fe66769", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313820", + "display": "Acetaminophen 160 MG Chewable Tablet" + } + ], + "text": "Acetaminophen 160 MG Chewable Tablet" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + }, + "authoredOn": "2018-06-08T23:41:28-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" + }, + "onBehalfOf": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:f2e15c0e-af89-4a65-9ce2-9e642f12416e", + "resource": { + "resourceType": "Claim", + "id": "f2e15c0e-af89-4a65-9ce2-9e642f12416e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "prescription": { + "reference": "urn:uuid:4573a9d7-d985-4d8c-815b-b8ce3fe66769" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + } + ] + } + ], + "total": { + "value": 3.2, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:500c7860-eb6d-4764-840b-c10727e1a251", + "resource": { + "resourceType": "CarePlan", + "id": "500c7860-eb6d-4764-840b-c10727e1a251", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225358003", + "display": "Wound care" + } + ], + "text": "Wound care" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + }, + "period": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-22T23:41:28-07:00" + }, + "addresses": [ + { + "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385949008", + "display": "Dressing change management" + } + ], + "text": "Dressing change management" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439830001", + "display": "Behavior to prevent infection" + } + ], + "text": "Behavior to prevent infection" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:78afde0b-9dd7-4f69-af1f-91ce8eb567ec", + "resource": { + "resourceType": "Claim", + "id": "78afde0b-9dd7-4f69-af1f-91ce8eb567ec", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "organization": { + "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:5baa99a7-6e14-4553-9a0e-7d425bc4b6cf" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "288086009" + } + ] + }, + "net": { + "value": 11608.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7286ec4f-02da-44da-9f1f-0e6ef6e332e1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7286ec4f-02da-44da-9f1f-0e6ef6e332e1", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "78afde0b-9dd7-4f69-af1f-91ce8eb567ec" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2018-06-09T00:53:28-07:00", + "end": "2019-06-09T00:53:28-07:00" + }, + "created": "2018-06-09T00:53:28-07:00", + "provider": { + "identifier": { + "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" + } + }, + "organization": { + "identifier": { + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:78afde0b-9dd7-4f69-af1f-91ce8eb567ec" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "288086009" + } + ] + }, + "servicedPeriod": { + "start": "2018-06-08T23:41:28-07:00", + "end": "2018-06-09T00:53:28-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "net": { + "value": 11608.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2321.746, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9286.984, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11608.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11608.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 9286.984, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e", + "resource": { + "resourceType": "Encounter", + "id": "eceec331-ba5c-4b19-b3e1-51c1faabf52e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + } + ], + "period": { + "start": "2019-05-16T23:41:28-07:00", + "end": "2019-05-16T23:56:28-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9c0aa340-be98-44b2-893f-64f2b106ae6c", + "resource": { + "resourceType": "Observation", + "id": "9c0aa340-be98-44b2-893f-64f2b106ae6c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 150.20, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b4afdce7-64fc-4214-9000-46763f503d55", + "resource": { + "resourceType": "Observation", + "id": "b4afdce7-64fc-4214-9000-46763f503d55", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a073353e-f81f-4b21-bbdf-2978a4b6f2bf", + "resource": { + "resourceType": "Observation", + "id": "a073353e-f81f-4b21-bbdf-2978a4b6f2bf", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 47.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1d15fa79-f80f-4f17-8a7a-4ecf0144e055", + "resource": { + "resourceType": "Observation", + "id": "1d15fa79-f80f-4f17-8a7a-4ecf0144e055", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 21.040, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:df2a62f1-a58e-4f82-b77d-83ab9a0e46ca", + "resource": { + "resourceType": "Observation", + "id": "df2a62f1-a58e-4f82-b77d-83ab9a0e46ca", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 85.914, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a8918824-a5b5-4196-be99-711cadd9c35e", + "resource": { + "resourceType": "Observation", + "id": "a8918824-a5b5-4196-be99-711cadd9c35e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 129, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0c5fb298-fca5-42c7-95a1-dbca00e3c7d8", + "resource": { + "resourceType": "Observation", + "id": "0c5fb298-fca5-42c7-95a1-dbca00e3c7d8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 68, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:35cb4ea6-e2fd-47ce-8a5f-e515f39398b5", + "resource": { + "resourceType": "Observation", + "id": "35cb4ea6-e2fd-47ce-8a5f-e515f39398b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9cfb44ff-4922-493b-9e66-fa1b6f269161", + "resource": { + "resourceType": "Observation", + "id": "9cfb44ff-4922-493b-9e66-fa1b6f269161", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 6.8759, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0033e9f6-3b36-4a23-8839-d727c750fa93", + "resource": { + "resourceType": "Observation", + "id": "0033e9f6-3b36-4a23-8839-d727c750fa93", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 4.0275, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3c46285f-0b7b-4bc6-be32-a2f9db82cca2", + "resource": { + "resourceType": "Observation", + "id": "3c46285f-0b7b-4bc6-be32-a2f9db82cca2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 14.612, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", + "resource": { + "resourceType": "Observation", + "id": "6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 47.912, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b589fc38-f806-41e6-8755-8f3abd9e2983", + "resource": { + "resourceType": "Observation", + "id": "b589fc38-f806-41e6-8755-8f3abd9e2983", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 92.859, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", + "resource": { + "resourceType": "Observation", + "id": "a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 32.219, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc612883-1917-4b60-9840-e7181c4a802f", + "resource": { + "resourceType": "Observation", + "id": "dc612883-1917-4b60-9840-e7181c4a802f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 35.773, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5f505a06-8b12-495a-83fd-7c666e256bad", + "resource": { + "resourceType": "Observation", + "id": "5f505a06-8b12-495a-83fd-7c666e256bad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 40.837, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bc93cd10-828d-4885-a907-66301769098b", + "resource": { + "resourceType": "Observation", + "id": "bc93cd10-828d-4885-a907-66301769098b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 312.35, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:33000cb4-08aa-4857-9870-cc14c045d611", + "resource": { + "resourceType": "Observation", + "id": "33000cb4-08aa-4857-9870-cc14c045d611", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 467.32, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7779ea12-b5d6-4ef5-bc39-34380458db93", + "resource": { + "resourceType": "Observation", + "id": "7779ea12-b5d6-4ef5-bc39-34380458db93", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueQuantity": { + "value": 9.9534, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b3938970-0a8f-46d9-96b5-46c96cf7d841", + "resource": { + "resourceType": "Observation", + "id": "b3938970-0a8f-46d9-96b5-46c96cf7d841", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f13170b9-4748-4c96-bd9f-f571aacfbe2e", + "resource": { + "resourceType": "Immunization", + "id": "f13170b9-4748-4c96-bd9f-f571aacfbe2e", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "115", + "display": "Tdap" + } + ], + "text": "Tdap" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "date": "2019-05-16T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:cd45550e-e9b8-4c57-9bc4-ee7c99462b6f", + "resource": { + "resourceType": "Immunization", + "id": "cd45550e-e9b8-4c57-9bc4-ee7c99462b6f", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "date": "2019-05-16T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:62808d09-f11f-4fa2-a367-b0c95444fc86", + "resource": { + "resourceType": "Immunization", + "id": "62808d09-f11f-4fa2-a367-b0c95444fc86", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "62", + "display": "HPV, quadrivalent" + } + ], + "text": "HPV, quadrivalent" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "date": "2019-05-16T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f4c97d17-a9ca-4464-aa6f-33c75ee5acbb", + "resource": { + "resourceType": "Immunization", + "id": "f4c97d17-a9ca-4464-aa6f-33c75ee5acbb", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "encounter": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "date": "2019-05-16T23:41:28-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:aa02cc24-70a4-4ee7-b7f0-6fa3f264e1a0", + "resource": { + "resourceType": "DiagnosticReport", + "id": "aa02cc24-70a4-4ee7-b7f0-6fa3f264e1a0", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "context": { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + }, + "effectiveDateTime": "2019-05-16T23:41:28-07:00", + "issued": "2019-05-16T23:41:28.175-07:00", + "result": [ + { + "reference": "urn:uuid:9cfb44ff-4922-493b-9e66-fa1b6f269161", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:0033e9f6-3b36-4a23-8839-d727c750fa93", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:3c46285f-0b7b-4bc6-be32-a2f9db82cca2", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:b589fc38-f806-41e6-8755-8f3abd9e2983", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:dc612883-1917-4b60-9840-e7181c4a802f", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:5f505a06-8b12-495a-83fd-7c666e256bad", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:bc93cd10-828d-4885-a907-66301769098b", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:33000cb4-08aa-4857-9870-cc14c045d611", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:7779ea12-b5d6-4ef5-bc39-34380458db93", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:7fbe2cfb-5891-41d9-bbd0-575b81fb2291", + "resource": { + "resourceType": "Claim", + "id": "7fbe2cfb-5891-41d9-bbd0-575b81fb2291", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "start": "2019-05-16T23:41:28-07:00", + "end": "2019-05-16T23:56:28-07:00" + }, + "organization": { + "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f13170b9-4748-4c96-bd9f-f571aacfbe2e" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:cd45550e-e9b8-4c57-9bc4-ee7c99462b6f" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:62808d09-f11f-4fa2-a367-b0c95444fc86" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:f4c97d17-a9ca-4464-aa6f-33c75ee5acbb" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8642d6ad-3f86-4afa-a804-2befa7a5deb3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8642d6ad-3f86-4afa-a804-2befa7a5deb3", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Cigna Health" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7fbe2cfb-5891-41d9-bbd0-575b81fb2291" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-05-16T23:56:28-07:00", + "end": "2020-05-16T23:56:28-07:00" + }, + "provider": { + "identifier": { + "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" + } + }, + "organization": { + "identifier": { + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 449.66400000000004, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json new file mode 100644 index 000000000000..ae6d3327430c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json @@ -0,0 +1,32061 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce", + "resource": { + "resourceType": "Patient", + "id": "3024090f-fe14-40a9-8fae-79952d3c95ce", + "text": { + "status": "generated", + "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 8703012480265391650 Population seed: 1586298193823
    " + }, + "extension": [ + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2106-3", + "display": "White" + } + }, + { + "url": "text", + "valueString": "White" + } + ] + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", + "extension": [ + { + "url": "ombCategory", + "valueCoding": { + "system": "urn:oid:2.16.840.1.113883.6.238", + "code": "2186-5", + "display": "Not Hispanic or Latino" + } + }, + { + "url": "text", + "valueString": "Not Hispanic or Latino" + } + ] + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", + "valueString": "Mary779 Green467" + }, + { + "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", + "valueCode": "F" + }, + { + "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", + "valueAddress": { + "city": "Danvers", + "state": "Massachusetts", + "country": "US" + } + }, + { + "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", + "valueDecimal": 0.0 + }, + { + "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", + "valueDecimal": 25.0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "MR", + "display": "Medical Record Number" + } + ], + "text": "Medical Record Number" + }, + "system": "http://hospital.smarthealthit.org", + "value": "3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/identifier-type", + "code": "SB", + "display": "Social Security Number" + } + ], + "text": "Social Security Number" + }, + "system": "http://hl7.org/fhir/sid/us-ssn", + "value": "999-50-6254" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "DL", + "display": "Driver's License" + } + ], + "text": "Driver's License" + }, + "system": "urn:oid:2.16.840.1.113883.4.3.25", + "value": "S99931895" + }, + { + "type": { + "coding": [ + { + "system": "http://hl7.org/fhir/v2/0203", + "code": "PPN", + "display": "Passport Number" + } + ], + "text": "Passport Number" + }, + "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", + "value": "X49969037X" + } + ], + "name": [ + { + "use": "official", + "family": "Hilll811", + "given": [ + "Shavonne800" + ], + "prefix": [ + "Ms." + ] + } + ], + "telecom": [ + { + "system": "phone", + "value": "555-145-6552", + "use": "home" + } + ], + "gender": "female", + "birthDate": "1994-09-06", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.173297866780715 + }, + { + "url": "longitude", + "valueDecimal": -72.21882892611714 + } + ] + } + ], + "line": [ + "253 Hackett Meadow Unit 91" + ], + "city": "Warren", + "state": "Massachusetts", + "country": "US" + } + ], + "maritalStatus": { + "coding": [ + { + "system": "http://hl7.org/fhir/v3/MaritalStatus", + "code": "S", + "display": "Never Married" + } + ], + "text": "Never Married" + }, + "multipleBirthInteger": 1, + "communication": [ + { + "language": { + "coding": [ + { + "system": "urn:ietf:bcp:47", + "code": "en-US", + "display": "English" + } + ], + "text": "English" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Patient" + } + }, + { + "fullUrl": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03", + "resource": { + "resourceType": "Organization", + "id": "5844ad77-f653-3c2b-b7dd-e97576ab3b03", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "BAYSTATE WING HOSPITAL AND MEDICAL CENTERS", + "telecom": [ + { + "system": "phone", + "value": "4132837651" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.187794 + }, + { + "url": "longitude", + "valueDecimal": -72.30846899999997 + } + ] + } + ], + "line": [ + "40 WRIGHT STREET" + ], + "city": "PALMER", + "state": "MA", + "postalCode": "01069", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9", + "resource": { + "resourceType": "Practitioner", + "id": "42f9935b-696e-37c9-8261-2a8c72d02dc9", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "130" + } + ], + "active": true, + "name": [ + { + "family": "Abernathy524", + "given": [ + "Kirby843" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "40 WRIGHT STREET" + ], + "city": "PALMER", + "state": "MA", + "postalCode": "01069", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5", + "resource": { + "resourceType": "Encounter", + "id": "1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2009-08-04T16:24:16-07:00", + "end": "2014-11-11T15:24:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:ee4121b9-5f8e-44d8-a33c-76b97b20fa75", + "resource": { + "resourceType": "Claim", + "id": "ee4121b9-5f8e-44d8-a33c-76b97b20fa75", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2009-08-04T16:24:16-07:00", + "end": "2014-11-11T15:24:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:26cbb6a0-9952-405f-9965-d638c86b19c9", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "26cbb6a0-9952-405f-9965-d638c86b19c9", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ee4121b9-5f8e-44d8-a33c-76b97b20fa75" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2015-11-11T15:24:16-08:00" + }, + "created": "2014-11-11T15:24:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:ee4121b9-5f8e-44d8-a33c-76b97b20fa75" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2009-08-04T16:24:16-07:00", + "end": "2014-11-11T15:24:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5" + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a", + "resource": { + "resourceType": "Encounter", + "id": "1bb4a6d8-4e87-448d-8765-0656cc53887a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2010-08-23T16:24:16-07:00", + "end": "2010-08-23T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a5cef09d-5e28-44fc-8c85-64ef2058260f", + "resource": { + "resourceType": "MedicationRequest", + "id": "a5cef09d-5e28-44fc-8c85-64ef2058260f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "748856", + "display": "Yaz 28 Day Pack" + } + ], + "text": "Yaz 28 Day Pack" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" + }, + "authoredOn": "2010-08-23T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:db886643-1469-4967-b699-399c05307706", + "resource": { + "resourceType": "Claim", + "id": "db886643-1469-4967-b699-399c05307706", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2010-08-23T16:24:16-07:00", + "end": "2010-08-23T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:a5cef09d-5e28-44fc-8c85-64ef2058260f" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" + } + ] + } + ], + "total": { + "value": 33.23, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b01e9050-0b65-4d8b-9695-76c3d2c055ee", + "resource": { + "resourceType": "Claim", + "id": "b01e9050-0b65-4d8b-9695-76c3d2c055ee", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2010-08-23T16:24:16-07:00", + "end": "2010-08-23T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3453fd2d-cd9c-462c-81a6-5e1062c46348", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3453fd2d-cd9c-462c-81a6-5e1062c46348", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b01e9050-0b65-4d8b-9695-76c3d2c055ee" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2010-08-23T16:39:16-07:00", + "end": "2011-08-23T16:39:16-07:00" + }, + "created": "2010-08-23T16:39:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:b01e9050-0b65-4d8b-9695-76c3d2c055ee" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2010-08-23T16:24:16-07:00", + "end": "2010-08-23T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4", + "resource": { + "resourceType": "Organization", + "id": "883860d6-0862-3a47-a18f-7256563c53f4", + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "HARRINGTON PHYSICIAN SERVICES INC", + "telecom": [ + { + "system": "phone", + "value": "413-245-0966" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.128176 + }, + { + "url": "longitude", + "valueDecimal": -72.205352 + } + ] + } + ], + "line": [ + "255 E OLD STURBRIDGE RD" + ], + "city": "BRIMFIELD", + "state": "MA", + "postalCode": "01010-9647", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915", + "resource": { + "resourceType": "Practitioner", + "id": "7212c355-6c66-3c09-b527-bb8d33b0a915", + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "85760" + } + ], + "active": true, + "name": [ + { + "family": "Kihn564", + "given": [ + "Hayden835" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "255 E OLD STURBRIDGE RD" + ], + "city": "BRIMFIELD", + "state": "MA", + "postalCode": "01010-9647", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:34954861-c50f-48d1-9751-45829b517826", + "resource": { + "resourceType": "Encounter", + "id": "34954861-c50f-48d1-9751-45829b517826", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2010-10-19T16:24:16-07:00", + "end": "2010-10-19T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b147e4ca-490a-4a11-91a8-d3df387ff13a", + "resource": { + "resourceType": "Observation", + "id": "b147e4ca-490a-4a11-91a8-d3df387ff13a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 159.80, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:29399f80-6863-4915-9942-bfff6fb83802", + "resource": { + "resourceType": "Observation", + "id": "29399f80-6863-4915-9942-bfff6fb83802", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32953059-5bb3-4e1e-913a-20f36e41d1e4", + "resource": { + "resourceType": "Observation", + "id": "32953059-5bb3-4e1e-913a-20f36e41d1e4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 47.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:85f2dcf5-0ce9-4f91-8056-340e74e7895f", + "resource": { + "resourceType": "Observation", + "id": "85f2dcf5-0ce9-4f91-8056-340e74e7895f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 18.590, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:446d7ae7-4543-419a-ace1-91ec13043ace", + "resource": { + "resourceType": "Observation", + "id": "446d7ae7-4543-419a-ace1-91ec13043ace", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 23.352, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:66f0f611-87c6-4004-bad8-93d20c19a174", + "resource": { + "resourceType": "Observation", + "id": "66f0f611-87c6-4004-bad8-93d20c19a174", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 120, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:43440109-3988-466b-82df-9af305f69234", + "resource": { + "resourceType": "Observation", + "id": "43440109-3988-466b-82df-9af305f69234", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 62, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:67bf611b-32b7-4007-86dc-670418f13b38", + "resource": { + "resourceType": "Observation", + "id": "67bf611b-32b7-4007-86dc-670418f13b38", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:78401336-35e4-47c5-be88-10d360390866", + "resource": { + "resourceType": "Observation", + "id": "78401336-35e4-47c5-be88-10d360390866", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 8.5499, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:472a834e-bd60-42ef-83b3-0aac2d7596fa", + "resource": { + "resourceType": "Observation", + "id": "472a834e-bd60-42ef-83b3-0aac2d7596fa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 4.4570, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", + "resource": { + "resourceType": "Observation", + "id": "25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 13.488, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cb528262-4f2b-4535-af32-e1cd91ede53b", + "resource": { + "resourceType": "Observation", + "id": "cb528262-4f2b-4535-af32-e1cd91ede53b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 44.643, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32a699de-77e5-4e01-85e6-8b1c5f2b04c2", + "resource": { + "resourceType": "Observation", + "id": "32a699de-77e5-4e01-85e6-8b1c5f2b04c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 89.460, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:511863d5-34ab-41f3-a5aa-5f15bab65932", + "resource": { + "resourceType": "Observation", + "id": "511863d5-34ab-41f3-a5aa-5f15bab65932", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 32.373, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8481b85d-8ee6-40c3-891f-289b3e682fad", + "resource": { + "resourceType": "Observation", + "id": "8481b85d-8ee6-40c3-891f-289b3e682fad", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 35.068, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:be946655-8e39-4e40-9f21-a2628b9438a9", + "resource": { + "resourceType": "Observation", + "id": "be946655-8e39-4e40-9f21-a2628b9438a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 39.197, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:efb11be8-0ea1-40ac-84ce-443106143724", + "resource": { + "resourceType": "Observation", + "id": "efb11be8-0ea1-40ac-84ce-443106143724", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 331.86, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:2646f8a3-8c1d-4908-98d6-9540d57246b2", + "resource": { + "resourceType": "Observation", + "id": "2646f8a3-8c1d-4908-98d6-9540d57246b2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 256.08, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:12ccbcb3-76c0-4d8d-966b-0bede82b17fe", + "resource": { + "resourceType": "Observation", + "id": "12ccbcb3-76c0-4d8d-966b-0bede82b17fe", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueQuantity": { + "value": 10.098, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0cec3548-cd55-4b06-884f-47e45297afb1", + "resource": { + "resourceType": "Observation", + "id": "0cec3548-cd55-4b06-884f-47e45297afb1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c89ea1ae-21e5-4af9-8c74-770c20bca76b", + "resource": { + "resourceType": "Immunization", + "id": "c89ea1ae-21e5-4af9-8c74-770c20bca76b", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "date": "2010-10-19T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e8092f33-962e-425d-82a4-5c88cf5621cd", + "resource": { + "resourceType": "Immunization", + "id": "e8092f33-962e-425d-82a4-5c88cf5621cd", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "date": "2010-10-19T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:4fc2be12-efb2-40b0-9c96-7505833a0f4b", + "resource": { + "resourceType": "DiagnosticReport", + "id": "4fc2be12-efb2-40b0-9c96-7505833a0f4b", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + }, + "effectiveDateTime": "2010-10-19T16:24:16-07:00", + "issued": "2010-10-19T16:24:16.644-07:00", + "result": [ + { + "reference": "urn:uuid:78401336-35e4-47c5-be88-10d360390866", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:472a834e-bd60-42ef-83b3-0aac2d7596fa", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:cb528262-4f2b-4535-af32-e1cd91ede53b", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:32a699de-77e5-4e01-85e6-8b1c5f2b04c2", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:511863d5-34ab-41f3-a5aa-5f15bab65932", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:8481b85d-8ee6-40c3-891f-289b3e682fad", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:be946655-8e39-4e40-9f21-a2628b9438a9", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:efb11be8-0ea1-40ac-84ce-443106143724", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:2646f8a3-8c1d-4908-98d6-9540d57246b2", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:12ccbcb3-76c0-4d8d-966b-0bede82b17fe", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:9b8f71ed-15d7-419b-81cd-71c2e0643795", + "resource": { + "resourceType": "Claim", + "id": "9b8f71ed-15d7-419b-81cd-71c2e0643795", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2010-10-19T16:24:16-07:00", + "end": "2010-10-19T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:c89ea1ae-21e5-4af9-8c74-770c20bca76b" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e8092f33-962e-425d-82a4-5c88cf5621cd" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:f1244e38-710f-44fd-b2fa-383f2331fc28", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "f1244e38-710f-44fd-b2fa-383f2331fc28", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9b8f71ed-15d7-419b-81cd-71c2e0643795" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2010-10-19T16:39:16-07:00", + "end": "2011-10-19T16:39:16-07:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 224.83200000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a", + "resource": { + "resourceType": "Encounter", + "id": "4c5f6878-6daa-483c-a0eb-1f36da49417a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2011-08-18T16:24:16-07:00", + "end": "2011-08-18T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e680310c-0089-4f8b-bb75-0e53f6289636", + "resource": { + "resourceType": "MedicationRequest", + "id": "e680310c-0089-4f8b-bb75-0e53f6289636", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "978950", + "display": "Natazia 28 Day Pack" + } + ], + "text": "Natazia 28 Day Pack" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" + }, + "authoredOn": "2011-08-18T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:6e400c21-7cb8-41be-b543-cef90ca4c662", + "resource": { + "resourceType": "Claim", + "id": "6e400c21-7cb8-41be-b543-cef90ca4c662", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2011-08-18T16:24:16-07:00", + "end": "2011-08-18T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:e680310c-0089-4f8b-bb75-0e53f6289636" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" + } + ] + } + ], + "total": { + "value": 46.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d96cab55-0542-418e-b436-7ca1718f8c5b", + "resource": { + "resourceType": "Claim", + "id": "d96cab55-0542-418e-b436-7ca1718f8c5b", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2011-08-18T16:24:16-07:00", + "end": "2011-08-18T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6c15bc73-870d-4ce6-80c1-bd60dc1bbb78", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6c15bc73-870d-4ce6-80c1-bd60dc1bbb78", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d96cab55-0542-418e-b436-7ca1718f8c5b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2011-08-18T16:39:16-07:00", + "end": "2012-08-18T16:39:16-07:00" + }, + "created": "2011-08-18T16:39:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:d96cab55-0542-418e-b436-7ca1718f8c5b" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2011-08-18T16:24:16-07:00", + "end": "2011-08-18T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022", + "resource": { + "resourceType": "Encounter", + "id": "b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "410620009", + "display": "Well child visit (procedure)" + } + ], + "text": "Well child visit (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2011-10-25T16:24:16-07:00", + "end": "2011-10-25T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b266256f-9b16-48c5-8730-35e8f34cf460", + "resource": { + "resourceType": "Observation", + "id": "b266256f-9b16-48c5-8730-35e8f34cf460", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 160.10, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a54bb9ba-7de2-4b82-b271-8b2eef095de8", + "resource": { + "resourceType": "Observation", + "id": "a54bb9ba-7de2-4b82-b271-8b2eef095de8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b0e356ce-67e9-4dc1-a228-1ae10dd1a1a5", + "resource": { + "resourceType": "Observation", + "id": "b0e356ce-67e9-4dc1-a228-1ae10dd1a1a5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 46.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7cc12c6a-ae44-4626-b1b6-aed45de8ca5b", + "resource": { + "resourceType": "Observation", + "id": "7cc12c6a-ae44-4626-b1b6-aed45de8ca5b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 18.140, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0b89178a-8b44-4bf6-afe4-6462b9c790a9", + "resource": { + "resourceType": "Observation", + "id": "0b89178a-8b44-4bf6-afe4-6462b9c790a9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 12.593, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:244a1173-04bb-4cf9-9390-4cdf653ab566", + "resource": { + "resourceType": "Observation", + "id": "244a1173-04bb-4cf9-9390-4cdf653ab566", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 83, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 126, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:22277078-cea0-4664-be13-df195a78fc21", + "resource": { + "resourceType": "Observation", + "id": "22277078-cea0-4664-be13-df195a78fc21", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 85, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a3a897e-b6d2-4fc3-911e-42480e3a629b", + "resource": { + "resourceType": "Observation", + "id": "5a3a897e-b6d2-4fc3-911e-42480e3a629b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a615ac44-517a-46c1-8b2d-32606224a860", + "resource": { + "resourceType": "Observation", + "id": "a615ac44-517a-46c1-8b2d-32606224a860", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "effectiveDateTime": "2011-10-25T16:24:16-07:00", + "issued": "2011-10-25T16:24:16.644-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc890bd2-3e42-4fac-9cb5-631967c34989", + "resource": { + "resourceType": "Immunization", + "id": "fc890bd2-3e42-4fac-9cb5-631967c34989", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + }, + "date": "2011-10-25T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:f91dc1d7-0237-42ca-b1dc-4eee732f3af3", + "resource": { + "resourceType": "Claim", + "id": "f91dc1d7-0237-42ca-b1dc-4eee732f3af3", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2011-10-25T16:24:16-07:00", + "end": "2011-10-25T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:fc890bd2-3e42-4fac-9cb5-631967c34989" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:627db0a1-4778-4cc9-a0d5-949bce0e1471", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "627db0a1-4778-4cc9-a0d5-949bce0e1471", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f91dc1d7-0237-42ca-b1dc-4eee732f3af3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2011-10-25T16:39:16-07:00", + "end": "2012-10-25T16:39:16-07:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305", + "resource": { + "resourceType": "Encounter", + "id": "13af4cf2-ed03-4441-b836-db5066581305", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2012-10-30T16:24:16-07:00", + "end": "2012-10-30T16:54:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701", + "resource": { + "resourceType": "Condition", + "id": "42372dca-1642-40a5-acfa-8ecddbf32701", + "clinicalStatus": "active", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "59621000", + "display": "Hypertension" + } + ], + "text": "Hypertension" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "onsetDateTime": "2012-10-30T16:24:16-07:00", + "assertedDate": "2012-10-30T16:24:16-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:706460c4-3803-417a-8a50-df8c417fee87", + "resource": { + "resourceType": "Observation", + "id": "706460c4-3803-417a-8a50-df8c417fee87", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 160.30, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5748203a-0471-4992-8a6e-9a65e21897b7", + "resource": { + "resourceType": "Observation", + "id": "5748203a-0471-4992-8a6e-9a65e21897b7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8618ef2a-0916-41ac-917e-cb16989d6f7b", + "resource": { + "resourceType": "Observation", + "id": "8618ef2a-0916-41ac-917e-cb16989d6f7b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 44.900, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:68e86fb9-556c-4a89-8263-faa03fe0d921", + "resource": { + "resourceType": "Observation", + "id": "68e86fb9-556c-4a89-8263-faa03fe0d921", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 17.460, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:50b4c213-652c-4f14-a77a-097fffe2f2eb", + "resource": { + "resourceType": "Observation", + "id": "50b4c213-652c-4f14-a77a-097fffe2f2eb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 4.3434, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:87c89a21-dc08-4885-8f47-77ac9c5990fd", + "resource": { + "resourceType": "Observation", + "id": "87c89a21-dc08-4885-8f47-77ac9c5990fd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 99, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 173, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11b3b668-c3ac-4cf3-98dd-c5d54eea098e", + "resource": { + "resourceType": "Observation", + "id": "11b3b668-c3ac-4cf3-98dd-c5d54eea098e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 74, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5bed8478-2ac4-4bd4-94ae-cd2f07b71943", + "resource": { + "resourceType": "Observation", + "id": "5bed8478-2ac4-4bd4-94ae-cd2f07b71943", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b07409ec-7296-4831-838b-cbad2533a94d", + "resource": { + "resourceType": "Observation", + "id": "b07409ec-7296-4831-838b-cbad2533a94d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "effectiveDateTime": "2012-10-30T16:24:16-07:00", + "issued": "2012-10-30T16:24:16.644-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c4b229f-4426-4699-8239-f0246850fc51", + "resource": { + "resourceType": "Procedure", + "id": "7c4b229f-4426-4699-8239-f0246850fc51", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "performedPeriod": { + "start": "2012-10-30T16:24:16-07:00", + "end": "2012-10-30T16:39:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3ab5b202-9a2e-48d3-9bb9-3b7304217e4d", + "resource": { + "resourceType": "MedicationRequest", + "id": "3ab5b202-9a2e-48d3-9bb9-3b7304217e4d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "310798", + "display": "Hydrochlorothiazide 25 MG Oral Tablet" + } + ], + "text": "Hydrochlorothiazide 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "authoredOn": "2012-10-30T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:404c81af-98fb-44fe-878b-974fe397a1f6", + "resource": { + "resourceType": "Claim", + "id": "404c81af-98fb-44fe-878b-974fe397a1f6", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2012-10-30T16:24:16-07:00", + "end": "2012-10-30T16:54:16-07:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:3ab5b202-9a2e-48d3-9bb9-3b7304217e4d" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:faa7b756-3a02-48c5-bed2-4dbc5b952409", + "resource": { + "resourceType": "Immunization", + "id": "faa7b756-3a02-48c5-bed2-4dbc5b952409", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "date": "2012-10-30T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:df7f7291-36bd-4eda-92c8-43b22eccf514", + "resource": { + "resourceType": "CarePlan", + "id": "df7f7291-36bd-4eda-92c8-43b22eccf514", + "status": "active", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "443402002", + "display": "Lifestyle education regarding hypertension" + } + ], + "text": "Lifestyle education regarding hypertension" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + }, + "period": { + "start": "2012-10-30T16:24:16-07:00" + }, + "addresses": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "386463000", + "display": "Prescribed activity/exercise education" + } + ], + "text": "Prescribed activity/exercise education" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "413473000", + "display": "Counseling about alcohol consumption" + } + ], + "text": "Counseling about alcohol consumption" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "1151000175103", + "display": "Dietary approaches to stop hypertension diet" + } + ], + "text": "Dietary approaches to stop hypertension diet" + }, + "status": "in-progress" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "225323000", + "display": "Smoking cessation education" + } + ], + "text": "Smoking cessation education" + }, + "status": "in-progress" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:2ddfac78-0bc1-4978-9286-ee41d0c85454", + "resource": { + "resourceType": "Claim", + "id": "2ddfac78-0bc1-4978-9286-ee41d0c85454", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2012-10-30T16:24:16-07:00", + "end": "2012-10-30T16:54:16-07:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:faa7b756-3a02-48c5-bed2-4dbc5b952409" + } + } + ], + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7c4b229f-4426-4699-8239-f0246850fc51" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 421.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dc1460f2-e563-4161-b451-f88a9981cb05", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "dc1460f2-e563-4161-b451-f88a9981cb05", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "2ddfac78-0bc1-4978-9286-ee41d0c85454" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-10-30T16:54:16-07:00", + "end": "2013-10-30T16:54:16-07:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 84.34800000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 337.39200000000005, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 421.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 421.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 449.80800000000005, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de", + "resource": { + "resourceType": "Encounter", + "id": "2fae00ba-3d25-492e-908c-38d0475074de", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2012-11-29T15:24:16-08:00", + "end": "2012-11-29T15:39:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bdda12d8-b3c4-4415-9fd8-f467170240f5", + "resource": { + "resourceType": "Observation", + "id": "bdda12d8-b3c4-4415-9fd8-f467170240f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" + }, + "effectiveDateTime": "2012-11-29T15:24:16-08:00", + "issued": "2012-11-29T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 192, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:483cbb0e-c06e-4a0b-87b3-e9abadab17e2", + "resource": { + "resourceType": "MedicationRequest", + "id": "483cbb0e-c06e-4a0b-87b3-e9abadab17e2", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" + }, + "authoredOn": "2012-11-29T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0cd15c2c-68d9-4d64-b00a-40ce61e6bf16", + "resource": { + "resourceType": "Claim", + "id": "0cd15c2c-68d9-4d64-b00a-40ce61e6bf16", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2012-11-29T15:24:16-08:00", + "end": "2012-11-29T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:483cbb0e-c06e-4a0b-87b3-e9abadab17e2" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:917196aa-7ef3-478a-959b-b1afec00a7e0", + "resource": { + "resourceType": "Claim", + "id": "917196aa-7ef3-478a-959b-b1afec00a7e0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2012-11-29T15:24:16-08:00", + "end": "2012-11-29T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e3042db7-c3c7-42a3-adec-ff4ed4b57e0f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e3042db7-c3c7-42a3-adec-ff4ed4b57e0f", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "917196aa-7ef3-478a-959b-b1afec00a7e0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2012-11-29T15:39:16-08:00", + "end": "2013-11-29T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03", + "resource": { + "resourceType": "Encounter", + "id": "4f04379e-6a5e-4e7e-9898-c3391412cd03", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "390906007", + "display": "Hypertension follow-up encounter" + } + ], + "text": "Hypertension follow-up encounter" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2013-01-28T15:24:16-08:00", + "end": "2013-01-28T15:39:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:118f1665-cf5f-46b6-9464-c3b04239830e", + "resource": { + "resourceType": "Observation", + "id": "118f1665-cf5f-46b6-9464-c3b04239830e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03" + }, + "effectiveDateTime": "2013-01-28T15:24:16-08:00", + "issued": "2013-01-28T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5a7a60f1-45a9-485a-905c-73f129772ede", + "resource": { + "resourceType": "Claim", + "id": "5a7a60f1-45a9-485a-905c-73f129772ede", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2013-01-28T15:24:16-08:00", + "end": "2013-01-28T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a43f503c-90d6-45f8-b350-e5f40fa4c470", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a43f503c-90d6-45f8-b350-e5f40fa4c470", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5a7a60f1-45a9-485a-905c-73f129772ede" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-01-28T15:39:16-08:00", + "end": "2014-01-28T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0", + "resource": { + "resourceType": "Encounter", + "id": "d2c47f97-7465-4a1f-9a54-d2641f2e55b0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2013-11-05T15:24:16-08:00", + "end": "2013-11-05T15:39:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:fdb5362f-f3f9-4f15-9297-c70b7c7f655e", + "resource": { + "resourceType": "Observation", + "id": "fdb5362f-f3f9-4f15-9297-c70b7c7f655e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.40, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:014700a0-bd9a-458d-8276-fe76e28aab16", + "resource": { + "resourceType": "Observation", + "id": "014700a0-bd9a-458d-8276-fe76e28aab16", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b6981ec3-4498-4142-b63c-32960f654629", + "resource": { + "resourceType": "Observation", + "id": "b6981ec3-4498-4142-b63c-32960f654629", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 46.300, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c7403f81-5bc3-434c-bec3-08b6e13b3943", + "resource": { + "resourceType": "Observation", + "id": "c7403f81-5bc3-434c-bec3-08b6e13b3943", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 17.990, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d4996166-9094-4327-b4ff-ebdfd9bec972", + "resource": { + "resourceType": "Observation", + "id": "d4996166-9094-4327-b4ff-ebdfd9bec972", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 6.4173, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fda835e4-a508-42bb-8a8c-8f23e5e746b0", + "resource": { + "resourceType": "Observation", + "id": "fda835e4-a508-42bb-8a8c-8f23e5e746b0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 74, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 108, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f883ddde-e985-4fc4-b6d0-92e14a2d7782", + "resource": { + "resourceType": "Observation", + "id": "f883ddde-e985-4fc4-b6d0-92e14a2d7782", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 75, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e17d2564-008a-45ec-8876-04e292e2a4df", + "resource": { + "resourceType": "Observation", + "id": "e17d2564-008a-45ec-8876-04e292e2a4df", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4cdfb7e2-257d-40bd-9c4c-d065cf38367b", + "resource": { + "resourceType": "Observation", + "id": "4cdfb7e2-257d-40bd-9c4c-d065cf38367b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "effectiveDateTime": "2013-11-05T15:24:16-08:00", + "issued": "2013-11-05T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ef780297-3f41-4bca-be9b-1895b9cb2e9d", + "resource": { + "resourceType": "MedicationRequest", + "id": "ef780297-3f41-4bca-be9b-1895b9cb2e9d", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "authoredOn": "2013-11-05T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:d35d1d98-f587-4ead-8605-943efe14675d", + "resource": { + "resourceType": "Claim", + "id": "d35d1d98-f587-4ead-8605-943efe14675d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2013-11-05T15:24:16-08:00", + "end": "2013-11-05T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:ef780297-3f41-4bca-be9b-1895b9cb2e9d" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8480205d-e9b2-4df3-a9a6-b3d86e02f187", + "resource": { + "resourceType": "Immunization", + "id": "8480205d-e9b2-4df3-a9a6-b3d86e02f187", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + }, + "date": "2013-11-05T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877", + "resource": { + "resourceType": "Claim", + "id": "ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2013-11-05T15:24:16-08:00", + "end": "2013-11-05T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:8480205d-e9b2-4df3-a9a6-b3d86e02f187" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d480df78-9a3d-4ae4-9079-32c71f3d22a3", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d480df78-9a3d-4ae4-9079-32c71f3d22a3", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2013-11-05T15:39:16-08:00", + "end": "2014-11-05T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517", + "resource": { + "resourceType": "Encounter", + "id": "e67b8297-a732-4b85-9489-4837ff0fd517", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2014-03-18T16:24:16-07:00", + "end": "2014-03-18T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e4d261dc-f1fe-41aa-9ae0-852c85171753", + "resource": { + "resourceType": "MedicationRequest", + "id": "e4d261dc-f1fe-41aa-9ae0-852c85171753", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" + }, + "authoredOn": "2014-03-18T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:4c95154c-38d9-45be-9a4f-ad8789f56784", + "resource": { + "resourceType": "Claim", + "id": "4c95154c-38d9-45be-9a4f-ad8789f56784", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-18T16:24:16-07:00", + "end": "2014-03-18T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:e4d261dc-f1fe-41aa-9ae0-852c85171753" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:70d584d4-67c1-4761-8604-ee4308d7fa60", + "resource": { + "resourceType": "Immunization", + "id": "70d584d4-67c1-4761-8604-ee4308d7fa60", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" + }, + "date": "2014-03-18T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:887c3e1a-a14d-4d4b-81e9-49ab8aa67d46", + "resource": { + "resourceType": "Claim", + "id": "887c3e1a-a14d-4d4b-81e9-49ab8aa67d46", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-18T16:24:16-07:00", + "end": "2014-03-18T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:70d584d4-67c1-4761-8604-ee4308d7fa60" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:83341480-28ee-4f70-a1e1-c8c25e339fca", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "83341480-28ee-4f70-a1e1-c8c25e339fca", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "887c3e1a-a14d-4d4b-81e9-49ab8aa67d46" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-18T16:39:16-07:00", + "end": "2015-03-18T16:39:16-07:00" + }, + "created": "2014-03-18T16:39:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:887c3e1a-a14d-4d4b-81e9-49ab8aa67d46" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-03-18T16:24:16-07:00", + "end": "2014-03-18T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-03-18T16:24:16-07:00", + "end": "2014-03-18T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667", + "resource": { + "resourceType": "Encounter", + "id": "ee580418-5e8c-421b-8671-952e139f4667", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-03-13T17:00:16-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", + "resource": { + "resourceType": "Condition", + "id": "e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10509002", + "display": "Acute bronchitis (disorder)" + } + ], + "text": "Acute bronchitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "onsetDateTime": "2014-03-13T16:24:16-07:00", + "abatementDateTime": "2014-03-27T16:24:16-07:00", + "assertedDate": "2014-03-13T16:24:16-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:59fa98f4-ff20-4f7d-b532-fe647f635944", + "resource": { + "resourceType": "Observation", + "id": "59fa98f4-ff20-4f7d-b532-fe647f635944", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7917c548-f7c1-4b08-9cf7-7d23a63684ee", + "resource": { + "resourceType": "Observation", + "id": "7917c548-f7c1-4b08-9cf7-7d23a63684ee", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3b6f80c1-ad8f-4af7-9f84-65f56e096b9e", + "resource": { + "resourceType": "Observation", + "id": "3b6f80c1-ad8f-4af7-9f84-65f56e096b9e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 47.600, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:aeca7327-ba7a-49ad-ac01-80f694410961", + "resource": { + "resourceType": "Observation", + "id": "aeca7327-ba7a-49ad-ac01-80f694410961", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 18.480, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1107a2db-0c0c-4856-a71a-72ab01277117", + "resource": { + "resourceType": "Observation", + "id": "1107a2db-0c0c-4856-a71a-72ab01277117", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "59576-9", + "display": "Body mass index (BMI) [Percentile] Per age and gender" + } + ], + "text": "Body mass index (BMI) [Percentile] Per age and gender" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 10.111, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:80ebd06c-6695-4dac-b4ff-9682cf900ad3", + "resource": { + "resourceType": "Observation", + "id": "80ebd06c-6695-4dac-b4ff-9682cf900ad3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 76, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 113, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c7e5b77b-a63a-4cdb-90b0-512003ad8d91", + "resource": { + "resourceType": "Observation", + "id": "c7e5b77b-a63a-4cdb-90b0-512003ad8d91", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 90, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:685b71bf-1779-47d1-8925-95014ad1fbcc", + "resource": { + "resourceType": "Observation", + "id": "685b71bf-1779-47d1-8925-95014ad1fbcc", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueQuantity": { + "value": 15, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:98845d25-be1b-42e7-b8a2-bb208f0ec088", + "resource": { + "resourceType": "Observation", + "id": "98845d25-be1b-42e7-b8a2-bb208f0ec088", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "effectiveDateTime": "2014-03-18T16:24:16-07:00", + "issued": "2014-03-18T16:24:16.644-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8", + "resource": { + "resourceType": "Procedure", + "id": "b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "23426006", + "display": "Measurement of respiratory function (procedure)" + } + ], + "text": "Measurement of respiratory function (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "performedPeriod": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-03-13T16:45:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", + "display": "Acute bronchitis (disorder)" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:301853d7-80b5-4225-9be6-9ade82f136a6", + "resource": { + "resourceType": "MedicationRequest", + "id": "301853d7-80b5-4225-9be6-9ade82f136a6", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "313782", + "display": "Acetaminophen 325 MG Oral Tablet" + } + ], + "text": "Acetaminophen 325 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "authoredOn": "2014-03-13T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:fc18db14-4893-4b6f-931c-31c6ad741c82", + "resource": { + "resourceType": "Claim", + "id": "fc18db14-4893-4b6f-931c-31c6ad741c82", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-03-13T17:00:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:301853d7-80b5-4225-9be6-9ade82f136a6" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + } + ] + } + ], + "total": { + "value": 6.53, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:73008d30-b07d-4712-a95a-118b071da6a5", + "resource": { + "resourceType": "MedicationRequest", + "id": "73008d30-b07d-4712-a95a-118b071da6a5", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "authoredOn": "2014-03-18T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:1deb2f81-ab06-417a-aa2d-a426e2be697f", + "resource": { + "resourceType": "Claim", + "id": "1deb2f81-ab06-417a-aa2d-a426e2be697f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-03-13T17:00:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:73008d30-b07d-4712-a95a-118b071da6a5" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:93fafb6a-793e-42b4-ad9e-1b9a6f2c64bb", + "resource": { + "resourceType": "CarePlan", + "id": "93fafb6a-793e-42b4-ad9e-1b9a6f2c64bb", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "53950000", + "display": "Respiratory therapy" + } + ], + "text": "Respiratory therapy" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + }, + "period": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-11-11T15:24:16-08:00" + }, + "addresses": [ + { + "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "304510005", + "display": "Recommendation to avoid exercise" + } + ], + "text": "Recommendation to avoid exercise" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "371605008", + "display": "Deep breathing and coughing exercises" + } + ], + "text": "Deep breathing and coughing exercises" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:d9537751-3ae8-42b3-8e97-1a9732e549b9", + "resource": { + "resourceType": "Claim", + "id": "d9537751-3ae8-42b3-8e97-1a9732e549b9", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-03-13T16:24:16-07:00", + "end": "2014-03-13T17:00:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "23426006" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:95e255a3-d9e2-4451-a480-60b37e1a2d3b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "95e255a3-d9e2-4451-a480-60b37e1a2d3b", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d9537751-3ae8-42b3-8e97-1a9732e549b9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-03-13T17:00:16-07:00", + "end": "2015-03-13T17:00:16-07:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "23426006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c", + "resource": { + "resourceType": "Encounter", + "id": "26dbc751-7daf-4205-b8a6-0c74e6e5f58c", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:50:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:80d776f9-f6e7-40d5-9066-c79e84320359", + "resource": { + "resourceType": "Procedure", + "id": "80d776f9-f6e7-40d5-9066-c79e84320359", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" + }, + "performedPeriod": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:35:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3550163d-fd97-4910-bacf-490302fb4d6e", + "resource": { + "resourceType": "MedicationRequest", + "id": "3550163d-fd97-4910-bacf-490302fb4d6e", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1000126", + "display": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" + } + ], + "text": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" + }, + "authoredOn": "2014-08-02T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:5a93ecad-4365-48bc-ab30-9d313ea3599e", + "resource": { + "resourceType": "Claim", + "id": "5a93ecad-4365-48bc-ab30-9d313ea3599e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:50:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:3550163d-fd97-4910-bacf-490302fb4d6e" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" + } + ] + } + ], + "total": { + "value": 303.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5d45a5fa-8917-400b-8ee5-d6d9b6289b7d", + "resource": { + "resourceType": "Claim", + "id": "5d45a5fa-8917-400b-8ee5-d6d9b6289b7d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:50:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:80d776f9-f6e7-40d5-9066-c79e84320359" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3250.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d50d29e0-09d9-40fa-9283-0ccd3c5f3ea1", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d50d29e0-09d9-40fa-9283-0ccd3c5f3ea1", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "5d45a5fa-8917-400b-8ee5-d6d9b6289b7d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-08-02T16:50:16-07:00", + "end": "2015-08-02T16:50:16-07:00" + }, + "created": "2014-08-02T16:50:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:5d45a5fa-8917-400b-8ee5-d6d9b6289b7d" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2014-08-02T16:24:16-07:00", + "end": "2014-08-02T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3250.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 650.1500000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2600.6000000000004, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3250.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3250.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2600.6000000000004, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0", + "resource": { + "resourceType": "Encounter", + "id": "a1441018-28ac-4b9d-b110-edfe104c4df0", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:39:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:52324c9a-431e-4cc2-a2f2-9b54af5780c6", + "resource": { + "resourceType": "Immunization", + "id": "52324c9a-431e-4cc2-a2f2-9b54af5780c6", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "43", + "display": "Hep B, adult" + } + ], + "text": "Hep B, adult" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0" + }, + "date": "2014-11-11T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:79403ae7-90a0-433c-8059-beac56c514b4", + "resource": { + "resourceType": "Claim", + "id": "79403ae7-90a0-433c-8059-beac56c514b4", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:52324c9a-431e-4cc2-a2f2-9b54af5780c6" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3d4254cf-74c9-446c-80c6-b281f050f23e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "3d4254cf-74c9-446c-80c6-b281f050f23e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "79403ae7-90a0-433c-8059-beac56c514b4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2014-11-11T15:39:16-08:00", + "end": "2015-11-11T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce", + "resource": { + "resourceType": "Encounter", + "id": "72b477bd-f07b-4813-a790-779d1d0c10ce", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:eb4af27b-9ad1-4d3e-8392-6bdb9004cd10", + "resource": { + "resourceType": "Observation", + "id": "eb4af27b-9ad1-4d3e-8392-6bdb9004cd10", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:03ddccd2-7898-4cc8-8c2c-8ad4c2e8b23a", + "resource": { + "resourceType": "Observation", + "id": "03ddccd2-7898-4cc8-8c2c-8ad4c2e8b23a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 4, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:032a20e6-dd9e-4859-91ef-c9e07bb82d98", + "resource": { + "resourceType": "Observation", + "id": "032a20e6-dd9e-4859-91ef-c9e07bb82d98", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 49.5, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:84a95b87-d3b2-4e38-b3a8-773f301b322a", + "resource": { + "resourceType": "Observation", + "id": "84a95b87-d3b2-4e38-b3a8-773f301b322a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 19.230, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:496cc321-bdd0-4008-98df-12aa7607321c", + "resource": { + "resourceType": "Observation", + "id": "496cc321-bdd0-4008-98df-12aa7607321c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 80, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 121, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:431d7e02-6096-4c38-9bf4-435fe81560aa", + "resource": { + "resourceType": "Observation", + "id": "431d7e02-6096-4c38-9bf4-435fe81560aa", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 62, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ecd44b84-100a-4386-96d6-41d72d57c79a", + "resource": { + "resourceType": "Observation", + "id": "ecd44b84-100a-4386-96d6-41d72d57c79a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d8b1d755-363c-4f26-a223-debe4c5b636f", + "resource": { + "resourceType": "Observation", + "id": "d8b1d755-363c-4f26-a223-debe4c5b636f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "effectiveDateTime": "2014-11-11T15:24:16-08:00", + "issued": "2014-11-11T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc", + "resource": { + "resourceType": "Procedure", + "id": "c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "performedPeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:42:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:95a13d49-ad21-492d-92c4-07a2bf99f7c1", + "resource": { + "resourceType": "Procedure", + "id": "95a13d49-ad21-492d-92c4-07a2bf99f7c1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "performedPeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9fb1c25d-f3ad-4faf-9773-abb7d0dd575f", + "resource": { + "resourceType": "MedicationRequest", + "id": "9fb1c25d-f3ad-4faf-9773-abb7d0dd575f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + }, + "authoredOn": "2014-11-11T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:ada238df-b638-4073-a820-d854ae53aea0", + "resource": { + "resourceType": "Claim", + "id": "ada238df-b638-4073-a820-d854ae53aea0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:9fb1c25d-f3ad-4faf-9773-abb7d0dd575f" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b74ce645-0edd-4777-b74d-8ea43a007a7f", + "resource": { + "resourceType": "Claim", + "id": "b74ce645-0edd-4777-b74d-8ea43a007a7f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:95a13d49-ad21-492d-92c4-07a2bf99f7c1" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2006.87, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 2 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 679.38, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:37fa0a40-be1e-4fbb-aaaa-dbf56f81d422", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "37fa0a40-be1e-4fbb-aaaa-dbf56f81d422", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b74ce645-0edd-4777-b74d-8ea43a007a7f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2014-11-11T15:57:16-08:00", + "end": "2015-11-11T15:57:16-08:00" + }, + "created": "2014-11-11T15:57:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:b74ce645-0edd-4777-b74d-8ea43a007a7f" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2006.87, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 401.374, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1605.496, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2006.87, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2006.87, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "servicedPeriod": { + "start": "2014-11-11T15:24:16-08:00", + "end": "2014-11-11T15:57:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 679.38, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 135.876, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 543.504, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 679.38, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 679.38, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2149.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b", + "resource": { + "resourceType": "Encounter", + "id": "0053b566-62ae-4381-8276-dfe8329fa56b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "EMER" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "50849002", + "display": "Emergency room admission (procedure)" + } + ], + "text": "Emergency room admission (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0", + "resource": { + "resourceType": "Condition", + "id": "8943cf36-7c44-4885-8d3f-a42d6cb3d2f0", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "403191005", + "display": "Second degree burn" + } + ], + "text": "Second degree burn" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + }, + "onsetDateTime": "2015-01-10T15:24:16-08:00", + "abatementDateTime": "2015-02-07T15:24:16-08:00", + "assertedDate": "2015-01-10T15:24:16-08:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:19868572-9831-4f32-83a7-a0f4ace4a262", + "resource": { + "resourceType": "MedicationRequest", + "id": "19868572-9831-4f32-83a7-a0f4ace4a262", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "308192", + "display": "Amoxicillin 500 MG Oral Tablet" + } + ], + "text": "Amoxicillin 500 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + }, + "authoredOn": "2015-01-10T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a01b565e-672c-499a-aa3b-bfe1b6d3ad31", + "resource": { + "resourceType": "Claim", + "id": "a01b565e-672c-499a-aa3b-bfe1b6d3ad31", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:19868572-9831-4f32-83a7-a0f4ace4a262" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + } + ] + } + ], + "total": { + "value": 6.39, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:11e723d4-3ad4-45b3-b471-1658db7c1a11", + "resource": { + "resourceType": "MedicationRequest", + "id": "11e723d4-3ad4-45b3-b471-1658db7c1a11", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "861467", + "display": "Meperidine Hydrochloride 50 MG Oral Tablet" + } + ], + "text": "Meperidine Hydrochloride 50 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + }, + "authoredOn": "2015-01-10T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:128b2648-19a7-4f1c-aa7c-bc21c6ba0dec", + "resource": { + "resourceType": "Claim", + "id": "128b2648-19a7-4f1c-aa7c-bc21c6ba0dec", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:11e723d4-3ad4-45b3-b471-1658db7c1a11" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + } + ] + } + ], + "total": { + "value": 63.15, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:9bcbc8c6-32c2-45b6-9771-b0eca73d4eed", + "resource": { + "resourceType": "MedicationRequest", + "id": "9bcbc8c6-32c2-45b6-9771-b0eca73d4eed", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "310965", + "display": "Ibuprofen 200 MG Oral Tablet" + } + ], + "text": "Ibuprofen 200 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + }, + "authoredOn": "2015-01-10T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:57c10579-eadb-475f-a659-bacf204221fc", + "resource": { + "resourceType": "Claim", + "id": "57c10579-eadb-475f-a659-bacf204221fc", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:9bcbc8c6-32c2-45b6-9771-b0eca73d4eed" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + } + ] + } + ], + "total": { + "value": 8.6, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:12bf4c5c-b3d5-43fc-9cf6-40221abe93df", + "resource": { + "resourceType": "CarePlan", + "id": "12bf4c5c-b3d5-43fc-9cf6-40221abe93df", + "status": "completed", + "intent": "order", + "category": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "133901003", + "display": "Burn care" + } + ], + "text": "Burn care" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + }, + "period": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-02-07T15:24:16-08:00" + }, + "addresses": [ + { + "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" + } + ], + "activity": [ + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "385949008", + "display": "Dressing change management" + } + ], + "text": "Dressing change management" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "440381005", + "display": "Behavior to prevent sun exposure" + } + ], + "text": "Behavior to prevent sun exposure" + }, + "status": "completed" + } + }, + { + "detail": { + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "439830001", + "display": "Behavior to prevent infection" + } + ], + "text": "Behavior to prevent infection" + }, + "status": "completed" + } + } + ] + }, + "request": { + "method": "POST", + "url": "CarePlan" + } + }, + { + "fullUrl": "urn:uuid:13aa18f1-796b-4a35-818d-435b8c90eb7e", + "resource": { + "resourceType": "Claim", + "id": "13aa18f1-796b-4a35-818d-435b8c90eb7e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dbe71c77-bdff-4ee7-82fe-fad34fd39e34", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "dbe71c77-bdff-4ee7-82fe-fad34fd39e34", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "13aa18f1-796b-4a35-818d-435b8c90eb7e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-01-10T16:24:16-08:00", + "end": "2016-01-10T16:24:16-08:00" + }, + "created": "2015-01-10T16:24:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:13aa18f1-796b-4a35-818d-435b8c90eb7e" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-01-10T15:24:16-08:00", + "end": "2015-01-10T16:24:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "23", + "display": "Emergency Room" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e", + "resource": { + "resourceType": "Encounter", + "id": "5311a30f-25f2-44d6-baed-3db16bea760e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-02-10T15:24:16-08:00", + "end": "2015-02-10T15:49:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b7376f67-61a3-4b5a-9faf-2d33f9f288fd", + "resource": { + "resourceType": "Procedure", + "id": "b7376f67-61a3-4b5a-9faf-2d33f9f288fd", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" + }, + "performedPeriod": { + "start": "2015-02-10T15:24:16-08:00", + "end": "2015-02-10T15:34:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:e0a27f42-4002-4210-bc54-2938a529e4de", + "resource": { + "resourceType": "Claim", + "id": "e0a27f42-4002-4210-bc54-2938a529e4de", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-02-10T15:24:16-08:00", + "end": "2015-02-10T15:49:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b7376f67-61a3-4b5a-9faf-2d33f9f288fd" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 1595.30, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:dc8dbd3c-95d8-4fdd-8f52-e80c05d02005", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "dc8dbd3c-95d8-4fdd-8f52-e80c05d02005", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "e0a27f42-4002-4210-bc54-2938a529e4de" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-02-10T15:49:16-08:00", + "end": "2016-02-10T15:49:16-08:00" + }, + "created": "2015-02-10T15:49:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:e0a27f42-4002-4210-bc54-2938a529e4de" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-02-10T15:24:16-08:00", + "end": "2015-02-10T15:49:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2015-02-10T15:24:16-08:00", + "end": "2015-02-10T15:49:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 1595.30, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 319.06, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1276.24, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 1595.30, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 1595.30, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1276.24, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c99a397c-4270-4682-b9c6-9d41fb8d103e", + "resource": { + "resourceType": "Encounter", + "id": "c99a397c-4270-4682-b9c6-9d41fb8d103e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for 'check-up'" + } + ], + "text": "Encounter for 'check-up'" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-02-07T15:24:16-08:00", + "end": "2015-02-07T15:39:16-08:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "403191005", + "display": "Second degree burn" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:01d74f6b-c8d5-47e8-a3d9-4cde87513cc0", + "resource": { + "resourceType": "Claim", + "id": "01d74f6b-c8d5-47e8-a3d9-4cde87513cc0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-02-07T15:24:16-08:00", + "end": "2015-02-07T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c99a397c-4270-4682-b9c6-9d41fb8d103e" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:6bb085f1-4b6d-45bb-aef9-bb91dfc02c33", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "6bb085f1-4b6d-45bb-aef9-bb91dfc02c33", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "01d74f6b-c8d5-47e8-a3d9-4cde87513cc0" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-02-07T15:39:16-08:00", + "end": "2016-02-07T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99", + "resource": { + "resourceType": "Encounter", + "id": "836c52b1-575c-4bae-8128-68161b5edd99", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-05-12T16:24:16-07:00", + "end": "2015-05-12T16:57:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9", + "resource": { + "resourceType": "Procedure", + "id": "3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" + }, + "performedPeriod": { + "start": "2015-05-12T16:24:16-07:00", + "end": "2015-05-12T16:42:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:c5ee4911-c7e1-4706-b691-861b38c7b7aa", + "resource": { + "resourceType": "Claim", + "id": "c5ee4911-c7e1-4706-b691-861b38c7b7aa", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-05-12T16:24:16-07:00", + "end": "2015-05-12T16:57:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2063.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b458ca66-54b6-498b-ba2e-bb2e4b9a85b8", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b458ca66-54b6-498b-ba2e-bb2e4b9a85b8", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "c5ee4911-c7e1-4706-b691-861b38c7b7aa" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-05-12T16:57:16-07:00", + "end": "2016-05-12T16:57:16-07:00" + }, + "created": "2015-05-12T16:57:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:c5ee4911-c7e1-4706-b691-861b38c7b7aa" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-05-12T16:24:16-07:00", + "end": "2015-05-12T16:57:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2015-05-12T16:24:16-07:00", + "end": "2015-05-12T16:57:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2063.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 412.71400000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1650.8560000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2063.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2063.57, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1650.8560000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a", + "resource": { + "resourceType": "Encounter", + "id": "b4176cb3-6141-47ac-b482-802189b2436a", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-08-11T16:24:16-07:00", + "end": "2015-08-11T16:49:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a6316a8f-a8cd-4b87-a952-d1652bad4bb3", + "resource": { + "resourceType": "Procedure", + "id": "a6316a8f-a8cd-4b87-a952-d1652bad4bb3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" + }, + "performedPeriod": { + "start": "2015-08-11T16:24:16-07:00", + "end": "2015-08-11T16:34:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4d86806d-84f3-4543-815a-13164812a981", + "resource": { + "resourceType": "Claim", + "id": "4d86806d-84f3-4543-815a-13164812a981", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-08-11T16:24:16-07:00", + "end": "2015-08-11T16:49:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a6316a8f-a8cd-4b87-a952-d1652bad4bb3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2465.00, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:8abe3532-c650-407d-9fe7-4c0ba073557f", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "8abe3532-c650-407d-9fe7-4c0ba073557f", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4d86806d-84f3-4543-815a-13164812a981" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-08-11T16:49:16-07:00", + "end": "2016-08-11T16:49:16-07:00" + }, + "created": "2015-08-11T16:49:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:4d86806d-84f3-4543-815a-13164812a981" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-08-11T16:24:16-07:00", + "end": "2015-08-11T16:49:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2015-08-11T16:24:16-07:00", + "end": "2015-08-11T16:49:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2465.00, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 493.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1972.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2465.00, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2465.00, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1972.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6", + "resource": { + "resourceType": "Encounter", + "id": "f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2015-11-10T15:24:16-08:00", + "end": "2015-11-10T15:52:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:7377f015-4487-4c20-b347-1b5a0cd61530", + "resource": { + "resourceType": "Procedure", + "id": "7377f015-4487-4c20-b347-1b5a0cd61530", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" + }, + "performedPeriod": { + "start": "2015-11-10T15:24:16-08:00", + "end": "2015-11-10T15:37:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f76aa0d7-6f99-488d-bde3-bcec1bb5911d", + "resource": { + "resourceType": "Claim", + "id": "f76aa0d7-6f99-488d-bde3-bcec1bb5911d", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-11-10T15:24:16-08:00", + "end": "2015-11-10T15:52:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7377f015-4487-4c20-b347-1b5a0cd61530" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2859.59, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:95c1e424-54a2-4b85-9611-870f06608dcc", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "95c1e424-54a2-4b85-9611-870f06608dcc", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f76aa0d7-6f99-488d-bde3-bcec1bb5911d" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-11-10T15:52:16-08:00", + "end": "2016-11-10T15:52:16-08:00" + }, + "created": "2015-11-10T15:52:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:f76aa0d7-6f99-488d-bde3-bcec1bb5911d" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2015-11-10T15:24:16-08:00", + "end": "2015-11-10T15:52:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2015-11-10T15:24:16-08:00", + "end": "2015-11-10T15:52:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2859.59, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 571.918, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2287.672, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2859.59, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2859.59, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2287.672, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256", + "resource": { + "resourceType": "Encounter", + "id": "27b6fd4a-2a0e-4d82-a548-65d42540a256", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2015-11-17T15:24:16-08:00", + "end": "2015-11-17T15:54:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:56b7aea8-24fe-4fa5-aa4f-6a341129382a", + "resource": { + "resourceType": "Observation", + "id": "56b7aea8-24fe-4fa5-aa4f-6a341129382a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:921032c0-0114-4a5e-bbcd-0bf1c9bff709", + "resource": { + "resourceType": "Observation", + "id": "921032c0-0114-4a5e-bbcd-0bf1c9bff709", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:a5ef5443-2e42-4e09-ab85-df0c8135a613", + "resource": { + "resourceType": "Observation", + "id": "a5ef5443-2e42-4e09-ab85-df0c8135a613", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 51.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f38dcd03-99ee-4911-9507-722f574c5c15", + "resource": { + "resourceType": "Observation", + "id": "f38dcd03-99ee-4911-9507-722f574c5c15", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 19.940, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ed0e579a-37f5-40d6-a0b2-2e4e79d2980a", + "resource": { + "resourceType": "Observation", + "id": "ed0e579a-37f5-40d6-a0b2-2e4e79d2980a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 81, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 114, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e3246862-7a48-46e9-b4d7-884269f385c8", + "resource": { + "resourceType": "Observation", + "id": "e3246862-7a48-46e9-b4d7-884269f385c8", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 86, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0f3887f9-25d9-4779-bd9c-227a020e2156", + "resource": { + "resourceType": "Observation", + "id": "0f3887f9-25d9-4779-bd9c-227a020e2156", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 16, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", + "resource": { + "resourceType": "Observation", + "id": "76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "6690-2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Leukocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 6.2154, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:58080587-be34-4087-8da5-a0b5613a3e89", + "resource": { + "resourceType": "Observation", + "id": "58080587-be34-4087-8da5-a0b5613a3e89", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "789-8", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + } + ], + "text": "Erythrocytes [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 5.1053, + "unit": "10*6/uL", + "system": "http://unitsofmeasure.org", + "code": "10*6/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6510c26-22f1-4f8a-9771-61136a99039e", + "resource": { + "resourceType": "Observation", + "id": "c6510c26-22f1-4f8a-9771-61136a99039e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "718-7", + "display": "Hemoglobin [Mass/volume] in Blood" + } + ], + "text": "Hemoglobin [Mass/volume] in Blood" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 14.489, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c2ede517-f03d-4666-af16-bc6348570a06", + "resource": { + "resourceType": "Observation", + "id": "c2ede517-f03d-4666-af16-bc6348570a06", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "4544-3", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + } + ], + "text": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 45.437, + "unit": "%", + "system": "http://unitsofmeasure.org", + "code": "%" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:ee94ae44-9dde-459d-835e-568706fb71f5", + "resource": { + "resourceType": "Observation", + "id": "ee94ae44-9dde-459d-835e-568706fb71f5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "787-2", + "display": "MCV [Entitic volume] by Automated count" + } + ], + "text": "MCV [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 87.474, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:54220c6f-3fc7-49f3-a821-aff81184ed40", + "resource": { + "resourceType": "Observation", + "id": "54220c6f-3fc7-49f3-a821-aff81184ed40", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "785-6", + "display": "MCH [Entitic mass] by Automated count" + } + ], + "text": "MCH [Entitic mass] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 31.763, + "unit": "pg", + "system": "http://unitsofmeasure.org", + "code": "pg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:cd828386-be7e-4a31-9e67-06288795eade", + "resource": { + "resourceType": "Observation", + "id": "cd828386-be7e-4a31-9e67-06288795eade", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "786-4", + "display": "MCHC [Mass/volume] by Automated count" + } + ], + "text": "MCHC [Mass/volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 34.857, + "unit": "g/dL", + "system": "http://unitsofmeasure.org", + "code": "g/dL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:accf7740-d4c2-4546-81a6-eed772a2b91b", + "resource": { + "resourceType": "Observation", + "id": "accf7740-d4c2-4546-81a6-eed772a2b91b", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "21000-5", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + } + ], + "text": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 42.645, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7cd987ea-01e1-45bb-a84a-2bf12caa7595", + "resource": { + "resourceType": "Observation", + "id": "7cd987ea-01e1-45bb-a84a-2bf12caa7595", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "777-3", + "display": "Platelets [#/volume] in Blood by Automated count" + } + ], + "text": "Platelets [#/volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 280.68, + "unit": "10*3/uL", + "system": "http://unitsofmeasure.org", + "code": "10*3/uL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:548fcef3-5aeb-4aa9-9b6b-f30aee35c380", + "resource": { + "resourceType": "Observation", + "id": "548fcef3-5aeb-4aa9-9b6b-f30aee35c380", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32207-3", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 340.44, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:61153ae3-8d67-4ee3-ab0a-9173084a8bd3", + "resource": { + "resourceType": "Observation", + "id": "61153ae3-8d67-4ee3-ab0a-9173084a8bd3", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "laboratory", + "display": "laboratory" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "32623-1", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ], + "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueQuantity": { + "value": 9.5791, + "unit": "fL", + "system": "http://unitsofmeasure.org", + "code": "fL" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d1cfb57a-13d9-4773-b78c-4756dbc75cd5", + "resource": { + "resourceType": "Observation", + "id": "d1cfb57a-13d9-4773-b78c-4756dbc75cd5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7c54e167-8da3-427f-a62c-14691981f880", + "resource": { + "resourceType": "Procedure", + "id": "7c54e167-8da3-427f-a62c-14691981f880", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "performedPeriod": { + "start": "2015-11-17T15:24:16-08:00", + "end": "2015-11-17T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f6787a2a-9b22-4cb7-8309-de3dd002a848", + "resource": { + "resourceType": "MedicationRequest", + "id": "f6787a2a-9b22-4cb7-8309-de3dd002a848", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "authoredOn": "2015-11-17T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:7045b927-3b8e-4db5-9f50-981466a611e1", + "resource": { + "resourceType": "Claim", + "id": "7045b927-3b8e-4db5-9f50-981466a611e1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-11-17T15:24:16-08:00", + "end": "2015-11-17T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:f6787a2a-9b22-4cb7-8309-de3dd002a848" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:450ff0e1-86e6-41ad-8abf-eea174649235", + "resource": { + "resourceType": "Immunization", + "id": "450ff0e1-86e6-41ad-8abf-eea174649235", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "date": "2015-11-17T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:52c017da-3670-4ee4-8d3b-8cd949c53b99", + "resource": { + "resourceType": "Immunization", + "id": "52c017da-3670-4ee4-8d3b-8cd949c53b99", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "113", + "display": "Td (adult) preservative free" + } + ], + "text": "Td (adult) preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "date": "2015-11-17T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:20ecd3fa-f7b2-4eb7-b40a-0f344f060d04", + "resource": { + "resourceType": "Immunization", + "id": "20ecd3fa-f7b2-4eb7-b40a-0f344f060d04", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "43", + "display": "Hep B, adult" + } + ], + "text": "Hep B, adult" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "date": "2015-11-17T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:e226d9f5-cfe7-4ad6-a96b-efd5deb61440", + "resource": { + "resourceType": "Immunization", + "id": "e226d9f5-cfe7-4ad6-a96b-efd5deb61440", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "114", + "display": "meningococcal MCV4P" + } + ], + "text": "meningococcal MCV4P" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "date": "2015-11-17T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:72ddb507-ac2f-4a74-b1b0-0ef8eb4690be", + "resource": { + "resourceType": "DiagnosticReport", + "id": "72ddb507-ac2f-4a74-b1b0-0ef8eb4690be", + "status": "final", + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "58410-2", + "display": "Complete blood count (hemogram) panel - Blood by Automated count" + } + ], + "text": "Complete blood count (hemogram) panel - Blood by Automated count" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + }, + "effectiveDateTime": "2015-11-17T15:24:16-08:00", + "issued": "2015-11-17T15:24:16.644-08:00", + "result": [ + { + "reference": "urn:uuid:76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", + "display": "Leukocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:58080587-be34-4087-8da5-a0b5613a3e89", + "display": "Erythrocytes [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:c6510c26-22f1-4f8a-9771-61136a99039e", + "display": "Hemoglobin [Mass/volume] in Blood" + }, + { + "reference": "urn:uuid:c2ede517-f03d-4666-af16-bc6348570a06", + "display": "Hematocrit [Volume Fraction] of Blood by Automated count" + }, + { + "reference": "urn:uuid:ee94ae44-9dde-459d-835e-568706fb71f5", + "display": "MCV [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:54220c6f-3fc7-49f3-a821-aff81184ed40", + "display": "MCH [Entitic mass] by Automated count" + }, + { + "reference": "urn:uuid:cd828386-be7e-4a31-9e67-06288795eade", + "display": "MCHC [Mass/volume] by Automated count" + }, + { + "reference": "urn:uuid:accf7740-d4c2-4546-81a6-eed772a2b91b", + "display": "Erythrocyte distribution width [Entitic volume] by Automated count" + }, + { + "reference": "urn:uuid:7cd987ea-01e1-45bb-a84a-2bf12caa7595", + "display": "Platelets [#/volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:548fcef3-5aeb-4aa9-9b6b-f30aee35c380", + "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" + }, + { + "reference": "urn:uuid:61153ae3-8d67-4ee3-ab0a-9173084a8bd3", + "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" + } + ] + }, + "request": { + "method": "POST", + "url": "DiagnosticReport" + } + }, + { + "fullUrl": "urn:uuid:a1c6c4b5-0929-4205-ae1d-d1a13354df0e", + "resource": { + "resourceType": "Claim", + "id": "a1c6c4b5-0929-4205-ae1d-d1a13354df0e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2015-11-17T15:24:16-08:00", + "end": "2015-11-17T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:450ff0e1-86e6-41ad-8abf-eea174649235" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:52c017da-3670-4ee4-8d3b-8cd949c53b99" + } + }, + { + "sequence": 3, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:20ecd3fa-f7b2-4eb7-b40a-0f344f060d04" + } + }, + { + "sequence": 4, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e226d9f5-cfe7-4ad6-a96b-efd5deb61440" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:7c54e167-8da3-427f-a62c-14691981f880" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "informationLinkId": [ + 3 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 5, + "informationLinkId": [ + 4 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 6, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 569.72, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:aa3bad79-1435-488d-bc39-c64aee985a93", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "aa3bad79-1435-488d-bc39-c64aee985a93", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "a1c6c4b5-0929-4205-ae1d-d1a13354df0e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2015-11-17T15:54:16-08:00", + "end": "2016-11-17T15:54:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 5, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 6, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 113.94400000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 455.77600000000007, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 569.72, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 569.72, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 905.44, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e", + "resource": { + "resourceType": "Encounter", + "id": "7fa3c314-cfd9-4828-b940-09561aab021e", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2016-02-09T15:24:16-08:00", + "end": "2016-02-09T15:57:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3", + "resource": { + "resourceType": "Procedure", + "id": "4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" + }, + "performedPeriod": { + "start": "2016-02-09T15:24:16-08:00", + "end": "2016-02-09T15:42:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:bc44af79-8721-43c8-b2cd-47f08b4ee564", + "resource": { + "resourceType": "Claim", + "id": "bc44af79-8721-43c8-b2cd-47f08b4ee564", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-02-09T15:24:16-08:00", + "end": "2016-02-09T15:57:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2805.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:93b295ea-0e5e-4736-a82b-09e304a69900", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "93b295ea-0e5e-4736-a82b-09e304a69900", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bc44af79-8721-43c8-b2cd-47f08b4ee564" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-02-09T15:57:16-08:00", + "end": "2017-02-09T15:57:16-08:00" + }, + "created": "2016-02-09T15:57:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:bc44af79-8721-43c8-b2cd-47f08b4ee564" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-02-09T15:24:16-08:00", + "end": "2016-02-09T15:57:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2016-02-09T15:24:16-08:00", + "end": "2016-02-09T15:57:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2805.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 561.158, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2244.632, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2805.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2805.79, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2244.632, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733", + "resource": { + "resourceType": "Encounter", + "id": "5eefddac-8c8c-4e83-914a-c463a6bf3733", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2016-05-10T16:24:16-07:00", + "end": "2016-05-10T16:56:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:144c4c1a-24b0-456a-9604-812b5980254a", + "resource": { + "resourceType": "Procedure", + "id": "144c4c1a-24b0-456a-9604-812b5980254a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" + }, + "performedPeriod": { + "start": "2016-05-10T16:24:16-07:00", + "end": "2016-05-10T16:41:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:3d53c6c9-f912-4250-b82a-1b65fe7cafbc", + "resource": { + "resourceType": "Claim", + "id": "3d53c6c9-f912-4250-b82a-1b65fe7cafbc", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-05-10T16:24:16-07:00", + "end": "2016-05-10T16:56:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:144c4c1a-24b0-456a-9604-812b5980254a" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 1575.77, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:fa2f6cb4-4207-4fb9-a1ee-265e95e8dd6c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "fa2f6cb4-4207-4fb9-a1ee-265e95e8dd6c", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "3d53c6c9-f912-4250-b82a-1b65fe7cafbc" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-05-10T16:56:16-07:00", + "end": "2017-05-10T16:56:16-07:00" + }, + "created": "2016-05-10T16:56:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:3d53c6c9-f912-4250-b82a-1b65fe7cafbc" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-05-10T16:24:16-07:00", + "end": "2016-05-10T16:56:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2016-05-10T16:24:16-07:00", + "end": "2016-05-10T16:56:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 1575.77, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 315.154, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1260.616, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 1575.77, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 1575.77, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1260.616, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450", + "resource": { + "resourceType": "Encounter", + "id": "9813d114-5d0d-4ce2-859e-60ba0ee58450", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2016-08-09T16:24:16-07:00", + "end": "2016-08-09T16:57:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a3335838-52dc-499f-b85c-0c45567f4bf2", + "resource": { + "resourceType": "Procedure", + "id": "a3335838-52dc-499f-b85c-0c45567f4bf2", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" + }, + "performedPeriod": { + "start": "2016-08-09T16:24:16-07:00", + "end": "2016-08-09T16:42:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e", + "resource": { + "resourceType": "Claim", + "id": "bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-08-09T16:24:16-07:00", + "end": "2016-08-09T16:57:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a3335838-52dc-499f-b85c-0c45567f4bf2" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3977.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1fdca28d-41f6-4f6f-b1e7-ad5d55d314df", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "1fdca28d-41f6-4f6f-b1e7-ad5d55d314df", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-08-09T16:57:16-07:00", + "end": "2017-08-09T16:57:16-07:00" + }, + "created": "2016-08-09T16:57:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-08-09T16:24:16-07:00", + "end": "2016-08-09T16:57:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2016-08-09T16:24:16-07:00", + "end": "2016-08-09T16:57:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3977.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 795.578, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3182.312, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3977.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3977.89, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 3182.312, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8", + "resource": { + "resourceType": "Encounter", + "id": "84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2016-11-08T15:24:16-08:00", + "end": "2016-11-08T15:50:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bd84294e-72b9-43d4-9d1c-8d522ea2dbe4", + "resource": { + "resourceType": "Procedure", + "id": "bd84294e-72b9-43d4-9d1c-8d522ea2dbe4", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" + }, + "performedPeriod": { + "start": "2016-11-08T15:24:16-08:00", + "end": "2016-11-08T15:35:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:961295fd-ac28-4058-b1f3-23186fcc2b9b", + "resource": { + "resourceType": "Claim", + "id": "961295fd-ac28-4058-b1f3-23186fcc2b9b", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-11-08T15:24:16-08:00", + "end": "2016-11-08T15:50:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:bd84294e-72b9-43d4-9d1c-8d522ea2dbe4" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 4243.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a591055a-5f87-4a9f-b016-e1e65bffa0b4", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a591055a-5f87-4a9f-b016-e1e65bffa0b4", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "961295fd-ac28-4058-b1f3-23186fcc2b9b" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-11-08T15:50:16-08:00", + "end": "2017-11-08T15:50:16-08:00" + }, + "created": "2016-11-08T15:50:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:961295fd-ac28-4058-b1f3-23186fcc2b9b" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2016-11-08T15:24:16-08:00", + "end": "2016-11-08T15:50:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2016-11-08T15:24:16-08:00", + "end": "2016-11-08T15:50:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 4243.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 848.7, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3394.8, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 4243.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 4243.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 3394.8, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3", + "resource": { + "resourceType": "Encounter", + "id": "e5eb6cb2-888a-4514-850f-e46a29c504d3", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2016-11-22T15:24:16-08:00", + "end": "2016-11-22T15:54:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:cb28939a-d6a3-4a94-b106-13e799e173a5", + "resource": { + "resourceType": "Observation", + "id": "cb28939a-d6a3-4a94-b106-13e799e173a5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:3d65d4b1-2f4a-4df8-8355-1ead6e29c9b1", + "resource": { + "resourceType": "Observation", + "id": "3d65d4b1-2f4a-4df8-8355-1ead6e29c9b1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 1, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bfbbf58b-633a-4dc0-a7ec-460b2b428855", + "resource": { + "resourceType": "Observation", + "id": "bfbbf58b-633a-4dc0-a7ec-460b2b428855", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 53.400, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:fc51eaea-94b5-442a-b87d-f36f7f74b1f0", + "resource": { + "resourceType": "Observation", + "id": "fc51eaea-94b5-442a-b87d-f36f7f74b1f0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 20.720, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b59620b0-65f9-491d-a580-e8705a9df5b5", + "resource": { + "resourceType": "Observation", + "id": "b59620b0-65f9-491d-a580-e8705a9df5b5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 75, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 111, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:8dd5bb0c-07bb-427e-82a5-563a94d56fe5", + "resource": { + "resourceType": "Observation", + "id": "8dd5bb0c-07bb-427e-82a5-563a94d56fe5", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 94, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:39b3b48a-8113-4fdc-ac46-78240f2bd581", + "resource": { + "resourceType": "Observation", + "id": "39b3b48a-8113-4fdc-ac46-78240f2bd581", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9af92abd-2188-4a61-9139-c9817720dabb", + "resource": { + "resourceType": "Observation", + "id": "9af92abd-2188-4a61-9139-c9817720dabb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "effectiveDateTime": "2016-11-22T15:24:16-08:00", + "issued": "2016-11-22T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:f61c567d-61f5-4861-b182-b7572c2afcc5", + "resource": { + "resourceType": "Procedure", + "id": "f61c567d-61f5-4861-b182-b7572c2afcc5", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "performedPeriod": { + "start": "2016-11-22T15:24:16-08:00", + "end": "2016-11-22T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:7689a885-feeb-4a35-8dc8-f1f33e505142", + "resource": { + "resourceType": "MedicationRequest", + "id": "7689a885-feeb-4a35-8dc8-f1f33e505142", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "authoredOn": "2016-11-22T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:0f583b17-b21f-4c3c-ae68-87a36d2030b9", + "resource": { + "resourceType": "Claim", + "id": "0f583b17-b21f-4c3c-ae68-87a36d2030b9", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-11-22T15:24:16-08:00", + "end": "2016-11-22T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:7689a885-feeb-4a35-8dc8-f1f33e505142" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e632095a-f912-4a53-b609-67cded28f427", + "resource": { + "resourceType": "Immunization", + "id": "e632095a-f912-4a53-b609-67cded28f427", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "date": "2016-11-22T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:cbab968d-6970-468e-be4a-85ce1d351666", + "resource": { + "resourceType": "Immunization", + "id": "cbab968d-6970-468e-be4a-85ce1d351666", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "43", + "display": "Hep B, adult" + } + ], + "text": "Hep B, adult" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + }, + "date": "2016-11-22T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:ab8ebec2-c070-4d1d-8b08-0ce46cb970a4", + "resource": { + "resourceType": "Claim", + "id": "ab8ebec2-c070-4d1d-8b08-0ce46cb970a4", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2016-11-22T15:24:16-08:00", + "end": "2016-11-22T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:e632095a-f912-4a53-b609-67cded28f427" + } + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:cbab968d-6970-468e-be4a-85ce1d351666" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:f61c567d-61f5-4861-b182-b7572c2afcc5" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "informationLinkId": [ + 2 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 594.99, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a7864301-dcde-469a-acde-5e4a7d4e6a9e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a7864301-dcde-469a-acde-5e4a7d4e6a9e", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "ab8ebec2-c070-4d1d-8b08-0ce46cb970a4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2016-11-22T15:54:16-08:00", + "end": "2017-11-22T15:54:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 118.998, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 475.992, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 594.99, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 594.99, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 700.8240000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67", + "resource": { + "resourceType": "Encounter", + "id": "b80442d6-40f5-4adf-beb1-fc3d30d1fe67", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2017-02-07T15:24:16-08:00", + "end": "2017-02-07T15:58:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d29492cf-53cf-42f4-b9f3-57b0c3c2015c", + "resource": { + "resourceType": "Procedure", + "id": "d29492cf-53cf-42f4-b9f3-57b0c3c2015c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" + }, + "performedPeriod": { + "start": "2017-02-07T15:24:16-08:00", + "end": "2017-02-07T15:43:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9c4d60e1-87e9-49dc-8f48-9362358300ac", + "resource": { + "resourceType": "Claim", + "id": "9c4d60e1-87e9-49dc-8f48-9362358300ac", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-02-07T15:24:16-08:00", + "end": "2017-02-07T15:58:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:d29492cf-53cf-42f4-b9f3-57b0c3c2015c" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3400.19, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:79ebda33-0670-40cf-90e8-10c0f1f26022", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "79ebda33-0670-40cf-90e8-10c0f1f26022", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9c4d60e1-87e9-49dc-8f48-9362358300ac" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-02-07T15:58:16-08:00", + "end": "2018-02-07T15:58:16-08:00" + }, + "created": "2017-02-07T15:58:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:9c4d60e1-87e9-49dc-8f48-9362358300ac" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2017-02-07T15:24:16-08:00", + "end": "2017-02-07T15:58:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2017-02-07T15:24:16-08:00", + "end": "2017-02-07T15:58:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3400.19, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 680.038, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2720.152, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3400.19, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3400.19, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2720.152, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393", + "resource": { + "resourceType": "Encounter", + "id": "615d0f10-3f62-42c8-9fa3-400bc8adf393", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2017-05-09T16:24:16-07:00", + "end": "2017-05-09T16:50:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:529092e3-aef8-4964-bb61-a90be792e125", + "resource": { + "resourceType": "Procedure", + "id": "529092e3-aef8-4964-bb61-a90be792e125", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" + }, + "performedPeriod": { + "start": "2017-05-09T16:24:16-07:00", + "end": "2017-05-09T16:35:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:aa3f48d5-4407-40af-b52e-dd2e43ade642", + "resource": { + "resourceType": "Claim", + "id": "aa3f48d5-4407-40af-b52e-dd2e43ade642", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-05-09T16:24:16-07:00", + "end": "2017-05-09T16:50:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:529092e3-aef8-4964-bb61-a90be792e125" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3045.69, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d84ead3f-a1fb-4ce6-9fe9-0a170dc5dd36", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "d84ead3f-a1fb-4ce6-9fe9-0a170dc5dd36", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "aa3f48d5-4407-40af-b52e-dd2e43ade642" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-05-09T16:50:16-07:00", + "end": "2018-05-09T16:50:16-07:00" + }, + "created": "2017-05-09T16:50:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:aa3f48d5-4407-40af-b52e-dd2e43ade642" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2017-05-09T16:24:16-07:00", + "end": "2017-05-09T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2017-05-09T16:24:16-07:00", + "end": "2017-05-09T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3045.69, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 609.138, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2436.552, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3045.69, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3045.69, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2436.552, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf", + "resource": { + "resourceType": "Encounter", + "id": "e541275d-1165-4ae1-8cac-0cba7645c1cf", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:54:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5cc75830-38bd-43fd-bf7a-56a655e3f502", + "resource": { + "resourceType": "Procedure", + "id": "5cc75830-38bd-43fd-bf7a-56a655e3f502", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" + }, + "performedPeriod": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:39:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f59c2d32-a435-4454-82c6-71f297a11c3b", + "resource": { + "resourceType": "MedicationRequest", + "id": "f59c2d32-a435-4454-82c6-71f297a11c3b", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "1000126", + "display": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" + } + ], + "text": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" + }, + "authoredOn": "2017-07-17T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a559500c-b320-4ade-baca-39004ef8cc5f", + "resource": { + "resourceType": "Claim", + "id": "a559500c-b320-4ade-baca-39004ef8cc5f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:54:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:f59c2d32-a435-4454-82c6-71f297a11c3b" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" + } + ] + } + ], + "total": { + "value": 142.94, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:d7bf818e-1745-4237-9837-fb165ca6a864", + "resource": { + "resourceType": "Claim", + "id": "d7bf818e-1745-4237-9837-fb165ca6a864", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:54:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:5cc75830-38bd-43fd-bf7a-56a655e3f502" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2692.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:751b75ec-a610-4a43-95fe-2ab988d83746", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "751b75ec-a610-4a43-95fe-2ab988d83746", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "d7bf818e-1745-4237-9837-fb165ca6a864" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-07-17T16:54:16-07:00", + "end": "2018-07-17T16:54:16-07:00" + }, + "created": "2017-07-17T16:54:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:d7bf818e-1745-4237-9837-fb165ca6a864" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:54:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2017-07-17T16:24:16-07:00", + "end": "2017-07-17T16:54:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2692.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 538.5500000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2154.2000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2692.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2692.75, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2154.2000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111", + "resource": { + "resourceType": "Encounter", + "id": "e20727ff-80bb-4eb7-86c9-72397dcdb111", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2017-08-08T16:24:16-07:00", + "end": "2017-08-08T16:55:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:75e4692b-b185-431f-8925-5f2218b91b46", + "resource": { + "resourceType": "Procedure", + "id": "75e4692b-b185-431f-8925-5f2218b91b46", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" + }, + "performedPeriod": { + "start": "2017-08-08T16:24:16-07:00", + "end": "2017-08-08T16:40:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:99702b89-b1e1-48ed-a6a4-fdcbd52bcfec", + "resource": { + "resourceType": "Claim", + "id": "99702b89-b1e1-48ed-a6a4-fdcbd52bcfec", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-08-08T16:24:16-07:00", + "end": "2017-08-08T16:55:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:75e4692b-b185-431f-8925-5f2218b91b46" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 4099.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:a27f37ad-cb21-4eb1-a60b-4b92a224adeb", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "a27f37ad-cb21-4eb1-a60b-4b92a224adeb", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "99702b89-b1e1-48ed-a6a4-fdcbd52bcfec" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-08-08T16:55:16-07:00", + "end": "2018-08-08T16:55:16-07:00" + }, + "created": "2017-08-08T16:55:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:99702b89-b1e1-48ed-a6a4-fdcbd52bcfec" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2017-08-08T16:24:16-07:00", + "end": "2017-08-08T16:55:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2017-08-08T16:24:16-07:00", + "end": "2017-08-08T16:55:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 4099.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 819.884, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3279.536, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 4099.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 4099.42, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 3279.536, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7", + "resource": { + "resourceType": "Encounter", + "id": "8a77baac-ce39-4c76-9ac3-57e239fd23b7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2017-11-07T15:24:16-08:00", + "end": "2017-11-07T15:50:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:96ad8284-b274-40dd-a901-09da986d98c1", + "resource": { + "resourceType": "Procedure", + "id": "96ad8284-b274-40dd-a901-09da986d98c1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" + }, + "performedPeriod": { + "start": "2017-11-07T15:24:16-08:00", + "end": "2017-11-07T15:35:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:742d2a7d-cfa5-461e-9108-9c76a1d02850", + "resource": { + "resourceType": "Claim", + "id": "742d2a7d-cfa5-461e-9108-9c76a1d02850", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-11-07T15:24:16-08:00", + "end": "2017-11-07T15:50:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:96ad8284-b274-40dd-a901-09da986d98c1" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2659.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:0923210a-7eb1-4cc5-874e-4bfdeb6a3e7d", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "0923210a-7eb1-4cc5-874e-4bfdeb6a3e7d", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "742d2a7d-cfa5-461e-9108-9c76a1d02850" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-11-07T15:50:16-08:00", + "end": "2018-11-07T15:50:16-08:00" + }, + "created": "2017-11-07T15:50:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:742d2a7d-cfa5-461e-9108-9c76a1d02850" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2017-11-07T15:24:16-08:00", + "end": "2017-11-07T15:50:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2017-11-07T15:24:16-08:00", + "end": "2017-11-07T15:50:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2659.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 531.9399999999999, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2127.7599999999998, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2659.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2659.70, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2127.7599999999998, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534", + "resource": { + "resourceType": "Encounter", + "id": "c9a1750e-ebd2-47ea-ba82-07d15a39b534", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2017-11-28T15:24:16-08:00", + "end": "2017-11-28T15:54:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:eb5d23e2-00d6-494f-87f5-8b656648f32f", + "resource": { + "resourceType": "Observation", + "id": "eb5d23e2-00d6-494f-87f5-8b656648f32f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:0f750612-e979-4f7d-bf82-787f0c243e7d", + "resource": { + "resourceType": "Observation", + "id": "0f750612-e979-4f7d-bf82-787f0c243e7d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 0, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e015abf0-3f22-4cbe-bcde-8e6212e70955", + "resource": { + "resourceType": "Observation", + "id": "e015abf0-3f22-4cbe-bcde-8e6212e70955", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 54.800, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:32c5313c-c85d-4854-b8a0-9cc0d0c7c806", + "resource": { + "resourceType": "Observation", + "id": "32c5313c-c85d-4854-b8a0-9cc0d0c7c806", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 21.280, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:20bee56c-749a-4186-bc51-baf252fa2269", + "resource": { + "resourceType": "Observation", + "id": "20bee56c-749a-4186-bc51-baf252fa2269", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 88, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7abf6374-0c6d-4ff6-803b-581b89dbb400", + "resource": { + "resourceType": "Observation", + "id": "7abf6374-0c6d-4ff6-803b-581b89dbb400", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 79, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b37857cb-629a-4f48-bd63-45434d614f4f", + "resource": { + "resourceType": "Observation", + "id": "b37857cb-629a-4f48-bd63-45434d614f4f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:d6b6b044-8a6b-40e1-9507-beee3861f286", + "resource": { + "resourceType": "Observation", + "id": "d6b6b044-8a6b-40e1-9507-beee3861f286", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "effectiveDateTime": "2017-11-28T15:24:16-08:00", + "issued": "2017-11-28T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e6cfb446-a7cf-4a8f-b9ff-785266b3723a", + "resource": { + "resourceType": "Procedure", + "id": "e6cfb446-a7cf-4a8f-b9ff-785266b3723a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "performedPeriod": { + "start": "2017-11-28T15:24:16-08:00", + "end": "2017-11-28T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:96c51039-d8ec-401e-b298-c7e2883d1803", + "resource": { + "resourceType": "MedicationRequest", + "id": "96c51039-d8ec-401e-b298-c7e2883d1803", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "authoredOn": "2017-11-28T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:05f25804-496e-4831-9e06-ea2615b220e0", + "resource": { + "resourceType": "Claim", + "id": "05f25804-496e-4831-9e06-ea2615b220e0", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-11-28T15:24:16-08:00", + "end": "2017-11-28T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:96c51039-d8ec-401e-b298-c7e2883d1803" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:52069ac5-bab8-4ae7-849a-89dd783006c9", + "resource": { + "resourceType": "Immunization", + "id": "52069ac5-bab8-4ae7-849a-89dd783006c9", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + }, + "date": "2017-11-28T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:9cac16f9-504c-4e79-a964-9aaf2af3d090", + "resource": { + "resourceType": "Claim", + "id": "9cac16f9-504c-4e79-a964-9aaf2af3d090", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2017-11-28T15:24:16-08:00", + "end": "2017-11-28T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:52069ac5-bab8-4ae7-849a-89dd783006c9" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e6cfb446-a7cf-4a8f-b9ff-785266b3723a" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 828.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:e9b01422-3316-40cf-98a2-3f2e3fa294fe", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "e9b01422-3316-40cf-98a2-3f2e3fa294fe", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "9cac16f9-504c-4e79-a964-9aaf2af3d090" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2017-11-28T15:54:16-08:00", + "end": "2018-11-28T15:54:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 165.74800000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 662.9920000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 828.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 828.74, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 775.4080000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086", + "resource": { + "resourceType": "Encounter", + "id": "57d221cf-bfb3-4fa3-9bf9-684607766086", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2018-02-06T15:24:16-08:00", + "end": "2018-02-06T15:54:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:e64c2f4d-e72e-41d9-9b28-5c1519ee1641", + "resource": { + "resourceType": "Procedure", + "id": "e64c2f4d-e72e-41d9-9b28-5c1519ee1641", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" + }, + "performedPeriod": { + "start": "2018-02-06T15:24:16-08:00", + "end": "2018-02-06T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:b82ea338-9d3e-4346-a7e3-6e2d90607458", + "resource": { + "resourceType": "Claim", + "id": "b82ea338-9d3e-4346-a7e3-6e2d90607458", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-02-06T15:24:16-08:00", + "end": "2018-02-06T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:e64c2f4d-e72e-41d9-9b28-5c1519ee1641" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2069.04, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:c16b4bd1-b007-48c1-91b0-169ad84b331c", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "c16b4bd1-b007-48c1-91b0-169ad84b331c", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b82ea338-9d3e-4346-a7e3-6e2d90607458" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-02-06T15:54:16-08:00", + "end": "2019-02-06T15:54:16-08:00" + }, + "created": "2018-02-06T15:54:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:b82ea338-9d3e-4346-a7e3-6e2d90607458" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-02-06T15:24:16-08:00", + "end": "2018-02-06T15:54:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2018-02-06T15:24:16-08:00", + "end": "2018-02-06T15:54:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2069.04, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 413.808, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1655.232, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2069.04, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2069.04, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1655.232, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b", + "resource": { + "resourceType": "Encounter", + "id": "1f186cb0-f553-455d-bed0-dbe4a4bb627b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2018-05-08T16:24:16-07:00", + "end": "2018-05-08T16:58:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:b1ed392b-a682-4123-8c6f-e5615cf0b359", + "resource": { + "resourceType": "Procedure", + "id": "b1ed392b-a682-4123-8c6f-e5615cf0b359", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" + }, + "performedPeriod": { + "start": "2018-05-08T16:24:16-07:00", + "end": "2018-05-08T16:43:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:db932c55-16f3-4e3a-aa0d-4325537c066e", + "resource": { + "resourceType": "Claim", + "id": "db932c55-16f3-4e3a-aa0d-4325537c066e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-05-08T16:24:16-07:00", + "end": "2018-05-08T16:58:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b1ed392b-a682-4123-8c6f-e5615cf0b359" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3009.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:b6b44ca7-ecba-439a-b910-342cefa121f0", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "b6b44ca7-ecba-439a-b910-342cefa121f0", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "db932c55-16f3-4e3a-aa0d-4325537c066e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-05-08T16:58:16-07:00", + "end": "2019-05-08T16:58:16-07:00" + }, + "created": "2018-05-08T16:58:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:db932c55-16f3-4e3a-aa0d-4325537c066e" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-05-08T16:24:16-07:00", + "end": "2018-05-08T16:58:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2018-05-08T16:24:16-07:00", + "end": "2018-05-08T16:58:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3009.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 601.902, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2407.608, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3009.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3009.51, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2407.608, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580", + "resource": { + "resourceType": "Encounter", + "id": "1781a84e-5949-4dfb-9dfd-40a48141f580", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2018-08-07T16:24:16-07:00", + "end": "2018-08-07T16:58:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c3ff2f9d-fca8-4805-92df-6aecfa57de7b", + "resource": { + "resourceType": "Procedure", + "id": "c3ff2f9d-fca8-4805-92df-6aecfa57de7b", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" + }, + "performedPeriod": { + "start": "2018-08-07T16:24:16-07:00", + "end": "2018-08-07T16:43:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:61fde201-fafd-43a5-8b0f-5b224768af57", + "resource": { + "resourceType": "Claim", + "id": "61fde201-fafd-43a5-8b0f-5b224768af57", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-08-07T16:24:16-07:00", + "end": "2018-08-07T16:58:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:c3ff2f9d-fca8-4805-92df-6aecfa57de7b" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2889.55, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:66f77d02-e7f2-431c-8011-b07e16b0f128", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "66f77d02-e7f2-431c-8011-b07e16b0f128", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "61fde201-fafd-43a5-8b0f-5b224768af57" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-08-07T16:58:16-07:00", + "end": "2019-08-07T16:58:16-07:00" + }, + "created": "2018-08-07T16:58:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:61fde201-fafd-43a5-8b0f-5b224768af57" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-08-07T16:24:16-07:00", + "end": "2018-08-07T16:58:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2018-08-07T16:24:16-07:00", + "end": "2018-08-07T16:58:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2889.55, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 577.9100000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2311.6400000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2889.55, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2889.55, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2311.6400000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7", + "resource": { + "resourceType": "Encounter", + "id": "b0b85503-3074-400f-a1d6-841125065cd7", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2018-11-06T15:24:16-08:00", + "end": "2018-11-06T15:51:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:a7961d03-f6a4-4527-b9f8-82a3ce43a4f1", + "resource": { + "resourceType": "Procedure", + "id": "a7961d03-f6a4-4527-b9f8-82a3ce43a4f1", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" + }, + "performedPeriod": { + "start": "2018-11-06T15:24:16-08:00", + "end": "2018-11-06T15:36:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:4d2a50d9-dcba-43c1-8f79-4536334ea3fd", + "resource": { + "resourceType": "Claim", + "id": "4d2a50d9-dcba-43c1-8f79-4536334ea3fd", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-11-06T15:24:16-08:00", + "end": "2018-11-06T15:51:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:a7961d03-f6a4-4527-b9f8-82a3ce43a4f1" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2557.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:eeb3c89a-e04a-4bb4-9bfd-8ad97767939e", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "eeb3c89a-e04a-4bb4-9bfd-8ad97767939e", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "4d2a50d9-dcba-43c1-8f79-4536334ea3fd" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-11-06T15:51:16-08:00", + "end": "2019-11-06T15:51:16-08:00" + }, + "created": "2018-11-06T15:51:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:4d2a50d9-dcba-43c1-8f79-4536334ea3fd" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2018-11-06T15:24:16-08:00", + "end": "2018-11-06T15:51:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2018-11-06T15:24:16-08:00", + "end": "2018-11-06T15:51:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2557.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 511.58400000000006, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2046.3360000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2557.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2557.92, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2046.3360000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78", + "resource": { + "resourceType": "Encounter", + "id": "5ad6b6ab-bcec-49d0-808e-c621cb071b78", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2018-12-04T15:24:16-08:00", + "end": "2018-12-04T15:54:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:74dcf884-17e7-4201-ae3a-50b726d59399", + "resource": { + "resourceType": "Observation", + "id": "74dcf884-17e7-4201-ae3a-50b726d59399", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:82c9c963-030b-4d88-995f-6a1448e1501f", + "resource": { + "resourceType": "Observation", + "id": "82c9c963-030b-4d88-995f-6a1448e1501f", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:1a92fcbe-7551-4b93-9d5b-a4f3aed0585e", + "resource": { + "resourceType": "Observation", + "id": "1a92fcbe-7551-4b93-9d5b-a4f3aed0585e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 56.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4683224a-c9c1-4e1b-adb2-b57e7d80fd73", + "resource": { + "resourceType": "Observation", + "id": "4683224a-c9c1-4e1b-adb2-b57e7d80fd73", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 21.760, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:686678d7-24ec-4d18-a721-5d0e739be299", + "resource": { + "resourceType": "Observation", + "id": "686678d7-24ec-4d18-a721-5d0e739be299", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 79, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 130, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b5b9a122-bc57-4ecc-b76c-e9ba2011a5dd", + "resource": { + "resourceType": "Observation", + "id": "b5b9a122-bc57-4ecc-b76c-e9ba2011a5dd", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 92, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:dc56f53c-ed89-46f9-9a71-ad205769f5bb", + "resource": { + "resourceType": "Observation", + "id": "dc56f53c-ed89-46f9-9a71-ad205769f5bb", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:11716994-7b0a-4acb-9630-9132acb548c4", + "resource": { + "resourceType": "Observation", + "id": "11716994-7b0a-4acb-9630-9132acb548c4", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "effectiveDateTime": "2018-12-04T15:24:16-08:00", + "issued": "2018-12-04T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b9d8995f-9c1d-4b46-9be8-71a5e85370a7", + "resource": { + "resourceType": "Procedure", + "id": "b9d8995f-9c1d-4b46-9be8-71a5e85370a7", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "430193006", + "display": "Medication Reconciliation (procedure)" + } + ], + "text": "Medication Reconciliation (procedure)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "performedPeriod": { + "start": "2018-12-04T15:24:16-08:00", + "end": "2018-12-04T15:39:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:1ac6052c-2519-4964-b87e-a85881001e5f", + "resource": { + "resourceType": "MedicationRequest", + "id": "1ac6052c-2519-4964-b87e-a85881001e5f", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "authoredOn": "2018-12-04T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:07c2f84b-ec79-4aa0-8ca2-47240774cc8f", + "resource": { + "resourceType": "Claim", + "id": "07c2f84b-ec79-4aa0-8ca2-47240774cc8f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-12-04T15:24:16-08:00", + "end": "2018-12-04T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:1ac6052c-2519-4964-b87e-a85881001e5f" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:1e959667-3a05-4657-8b0b-d0acc8b83a4c", + "resource": { + "resourceType": "Immunization", + "id": "1e959667-3a05-4657-8b0b-d0acc8b83a4c", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + }, + "date": "2018-12-04T15:24:16-08:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9", + "resource": { + "resourceType": "Claim", + "id": "14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2018-12-04T15:24:16-08:00", + "end": "2018-12-04T15:54:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:1e959667-3a05-4657-8b0b-d0acc8b83a4c" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:b9d8995f-9c1d-4b46-9be8-71a5e85370a7" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "net": { + "value": 529.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:ad76f367-22da-4413-8d83-fb2ebf1ea597", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "ad76f367-22da-4413-8d83-fb2ebf1ea597", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2018-12-04T15:54:16-08:00", + "end": "2019-12-04T15:54:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "430193006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 105.94600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 423.78400000000005, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 529.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 529.73, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 536.2, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d", + "resource": { + "resourceType": "Encounter", + "id": "ca05aff0-7d51-4589-af05-4e7326ed294d", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-02-05T15:24:16-08:00", + "end": "2019-02-05T15:53:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:32839def-0a11-44ca-bc1f-3f4801db8b06", + "resource": { + "resourceType": "Procedure", + "id": "32839def-0a11-44ca-bc1f-3f4801db8b06", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" + }, + "performedPeriod": { + "start": "2019-02-05T15:24:16-08:00", + "end": "2019-02-05T15:38:16-08:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:f444b593-e10e-4763-acc9-c421fc99cad1", + "resource": { + "resourceType": "Claim", + "id": "f444b593-e10e-4763-acc9-c421fc99cad1", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-02-05T15:24:16-08:00", + "end": "2019-02-05T15:53:16-08:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:32839def-0a11-44ca-bc1f-3f4801db8b06" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 3612.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:359c91c3-b781-415e-ba82-c16c5b35d928", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "359c91c3-b781-415e-ba82-c16c5b35d928", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "f444b593-e10e-4763-acc9-c421fc99cad1" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-02-05T15:53:16-08:00", + "end": "2020-02-05T15:53:16-08:00" + }, + "created": "2019-02-05T15:53:16-08:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:f444b593-e10e-4763-acc9-c421fc99cad1" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-02-05T15:24:16-08:00", + "end": "2019-02-05T15:53:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2019-02-05T15:24:16-08:00", + "end": "2019-02-05T15:53:16-08:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 3612.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 722.5360000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 2890.1440000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 3612.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 3612.68, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 2890.1440000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478", + "resource": { + "resourceType": "Encounter", + "id": "3a830afb-f2b3-4513-ba44-546507458478", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185349003", + "display": "Encounter for check up (procedure)" + } + ], + "text": "Encounter for check up (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-04-16T16:24:16-07:00", + "end": "2019-04-16T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:d15936cd-4e20-41cd-987a-274987cb0798", + "resource": { + "resourceType": "MedicationRequest", + "id": "d15936cd-4e20-41cd-987a-274987cb0798", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" + }, + "authoredOn": "2019-04-16T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:b4d42175-b63d-4cc9-b1ad-5ff8fb929076", + "resource": { + "resourceType": "Claim", + "id": "b4d42175-b63d-4cc9-b1ad-5ff8fb929076", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-04-16T16:24:16-07:00", + "end": "2019-04-16T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:d15936cd-4e20-41cd-987a-274987cb0798" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:3e1963e2-6a19-45f2-97ed-db1bdc1a4c19", + "resource": { + "resourceType": "Immunization", + "id": "3e1963e2-6a19-45f2-97ed-db1bdc1a4c19", + "status": "completed", + "notGiven": false, + "vaccineCode": { + "coding": [ + { + "system": "http://hl7.org/fhir/sid/cvx", + "code": "140", + "display": "Influenza, seasonal, injectable, preservative free" + } + ], + "text": "Influenza, seasonal, injectable, preservative free" + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "encounter": { + "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" + }, + "date": "2019-04-16T16:24:16-07:00", + "primarySource": true + }, + "request": { + "method": "POST", + "url": "Immunization" + } + }, + { + "fullUrl": "urn:uuid:b0c38318-529d-4f3d-9b78-406a5adcc2e2", + "resource": { + "resourceType": "Claim", + "id": "b0c38318-529d-4f3d-9b78-406a5adcc2e2", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-04-16T16:24:16-07:00", + "end": "2019-04-16T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "information": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "http://hl7.org/fhir/claiminformationcategory", + "code": "info" + } + ] + }, + "valueReference": { + "reference": "urn:uuid:3e1963e2-6a19-45f2-97ed-db1bdc1a4c19" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:424c7c68-99c4-4424-a9f7-181a32af532b", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "424c7c68-99c4-4424-a9f7-181a32af532b", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "b0c38318-529d-4f3d-9b78-406a5adcc2e2" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-04-16T16:39:16-07:00", + "end": "2020-04-16T16:39:16-07:00" + }, + "created": "2019-04-16T16:39:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:b0c38318-529d-4f3d-9b78-406a5adcc2e2" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-04-16T16:24:16-07:00", + "end": "2019-04-16T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" + } + ] + }, + { + "sequence": 2, + "informationLinkId": [ + 1 + ], + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-04-16T16:24:16-07:00", + "end": "2019-04-16T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 28.104000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 140.52, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 112.41600000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4", + "resource": { + "resourceType": "Encounter", + "id": "42f30f56-7718-40f0-aaea-4af113c3b0d4", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "185345009", + "display": "Encounter for symptom" + } + ], + "text": "Encounter for symptom" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-04-13T16:24:16-07:00", + "end": "2019-04-13T16:39:16-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3", + "resource": { + "resourceType": "Condition", + "id": "9f73c057-cb43-4696-860d-2c5b3b876af3", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "195662009", + "display": "Acute viral pharyngitis (disorder)" + } + ], + "text": "Acute viral pharyngitis (disorder)" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "onsetDateTime": "2019-04-13T16:24:16-07:00", + "abatementDateTime": "2019-04-21T16:24:16-07:00", + "assertedDate": "2019-04-13T16:24:16-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:b1d1927a-d7f3-4113-a2da-ae978c9bf38d", + "resource": { + "resourceType": "Observation", + "id": "b1d1927a-d7f3-4113-a2da-ae978c9bf38d", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8310-5", + "display": "Body temperature" + } + ], + "text": "Body temperature" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-13T16:24:16-07:00", + "issued": "2019-04-13T16:24:16.644-07:00", + "valueQuantity": { + "value": 37.342, + "unit": "Cel", + "system": "http://unitsofmeasure.org", + "code": "Cel" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:01882e32-4618-4fd9-ae85-80148f8deb70", + "resource": { + "resourceType": "Observation", + "id": "01882e32-4618-4fd9-ae85-80148f8deb70", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:e55a0b1c-0639-433b-8ae4-81a769e287c2", + "resource": { + "resourceType": "Observation", + "id": "e55a0b1c-0639-433b-8ae4-81a769e287c2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 3, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:c6c879d6-88c2-4b76-b930-526d105adff0", + "resource": { + "resourceType": "Observation", + "id": "c6c879d6-88c2-4b76-b930-526d105adff0", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 56.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4c8379ca-3c71-47bc-a58f-a39e52dfe557", + "resource": { + "resourceType": "Observation", + "id": "4c8379ca-3c71-47bc-a58f-a39e52dfe557", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 21.760, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bd8773a8-709c-4ba9-b925-979b503aabc7", + "resource": { + "resourceType": "Observation", + "id": "bd8773a8-709c-4ba9-b925-979b503aabc7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 86, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 123, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:4f5cf999-facf-4070-b6cf-0dd62694c75a", + "resource": { + "resourceType": "Observation", + "id": "4f5cf999-facf-4070-b6cf-0dd62694c75a", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 68, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:19a7a129-4e38-4e3b-9c04-dedc560012f1", + "resource": { + "resourceType": "Observation", + "id": "19a7a129-4e38-4e3b-9c04-dedc560012f1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueQuantity": { + "value": 13, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:7e32bdd2-0c04-4b3d-a3db-7f7466f0f318", + "resource": { + "resourceType": "Observation", + "id": "7e32bdd2-0c04-4b3d-a3db-7f7466f0f318", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "effectiveDateTime": "2019-04-16T16:24:16-07:00", + "issued": "2019-04-16T16:24:16.644-07:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:88506957-24fe-4807-8326-7e2c50a75998", + "resource": { + "resourceType": "MedicationRequest", + "id": "88506957-24fe-4807-8326-7e2c50a75998", + "status": "stopped", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + }, + "authoredOn": "2019-04-16T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:a1aa0150-cc31-4919-b243-8177710d1f1c", + "resource": { + "resourceType": "Claim", + "id": "a1aa0150-cc31-4919-b243-8177710d1f1c", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-04-13T16:24:16-07:00", + "end": "2019-04-13T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:88506957-24fe-4807-8326-7e2c50a75998" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7f1957a1-1522-4d2f-858a-ef575ab635e4", + "resource": { + "resourceType": "Claim", + "id": "7f1957a1-1522-4d2f-858a-ef575ab635e4", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-04-13T16:24:16-07:00", + "end": "2019-04-13T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:73a962fd-c8b2-40b2-86a2-3d465e653def", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "73a962fd-c8b2-40b2-86a2-3d465e653def", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "7f1957a1-1522-4d2f-858a-ef575ab635e4" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-04-13T16:39:16-07:00", + "end": "2020-04-13T16:39:16-07:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce", + "resource": { + "resourceType": "Encounter", + "id": "c5353211-363e-4740-8d26-163cd81779ce", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "308335008", + "display": "Patient encounter procedure" + } + ], + "text": "Patient encounter procedure" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-05-07T16:24:16-07:00", + "end": "2019-05-07T16:50:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:bd16401f-27a3-4e93-875d-848fcc226d0d", + "resource": { + "resourceType": "Procedure", + "id": "bd16401f-27a3-4e93-875d-848fcc226d0d", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "76601001", + "display": "Intramuscular injection" + } + ], + "text": "Intramuscular injection" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" + }, + "performedPeriod": { + "start": "2019-05-07T16:24:16-07:00", + "end": "2019-05-07T16:35:16-07:00" + } + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:daef21e7-4157-4cc9-974b-8bcce6742a94", + "resource": { + "resourceType": "Claim", + "id": "daef21e7-4157-4cc9-974b-8bcce6742a94", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-05-07T16:24:16-07:00", + "end": "2019-05-07T16:50:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:bd16401f-27a3-4e93-875d-848fcc226d0d" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "net": { + "value": 2137.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:54b60604-c463-4a6c-86d3-b1c1cc9219b5", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "54b60604-c463-4a6c-86d3-b1c1cc9219b5", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "daef21e7-4157-4cc9-974b-8bcce6742a94" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-05-07T16:50:16-07:00", + "end": "2020-05-07T16:50:16-07:00" + }, + "created": "2019-05-07T16:50:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:daef21e7-4157-4cc9-974b-8bcce6742a94" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-05-07T16:24:16-07:00", + "end": "2019-05-07T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" + } + ] + }, + { + "sequence": 2, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "service": { + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "76601001" + } + ] + }, + "servicedPeriod": { + "start": "2019-05-07T16:24:16-07:00", + "end": "2019-05-07T16:50:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "net": { + "value": 2137.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 427.53400000000005, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 1710.1360000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 2137.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 2137.67, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + }, + "reason": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "code": "A", + "display": "Allowed" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1710.1360000000002, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416", + "resource": { + "resourceType": "Encounter", + "id": "ae6e038e-9e72-45a4-ab30-6f0a24fbd416", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424441002", + "display": "Prenatal initial visit" + } + ], + "text": "Prenatal initial visit" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-08-13T16:24:16-07:00", + "end": "2019-08-13T17:09:16-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "resource": { + "resourceType": "Condition", + "id": "c9731186-6731-4a66-82e2-0669427caeb2", + "clinicalStatus": "resolved", + "verificationStatus": "confirmed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ], + "text": "Normal pregnancy" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" + }, + "onsetDateTime": "2019-08-13T16:24:16-07:00", + "abatementDateTime": "2019-08-27T16:24:16-07:00", + "assertedDate": "2019-08-13T16:24:16-07:00" + }, + "request": { + "method": "POST", + "url": "Condition" + } + }, + { + "fullUrl": "urn:uuid:4d516bf3-2917-46e6-87be-51264b421e30", + "resource": { + "resourceType": "Procedure", + "id": "4d516bf3-2917-46e6-87be-51264b421e30", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "252160004", + "display": "Standard pregnancy test" + } + ], + "text": "Standard pregnancy test" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" + }, + "performedPeriod": { + "start": "2019-08-13T16:24:16-07:00", + "end": "2019-08-13T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:9f541dc0-650b-4558-a996-6e7368c0ec72", + "resource": { + "resourceType": "Procedure", + "id": "9f541dc0-650b-4558-a996-6e7368c0ec72", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "169230002", + "display": "Ultrasound scan for fetal viability" + } + ], + "text": "Ultrasound scan for fetal viability" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" + }, + "performedPeriod": { + "start": "2019-08-13T16:24:16-07:00", + "end": "2019-08-13T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:80d606df-cf79-487f-8e24-ca7b42363cee", + "resource": { + "resourceType": "Claim", + "id": "80d606df-cf79-487f-8e24-ca7b42363cee", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-13T16:24:16-07:00", + "end": "2019-08-13T17:09:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "diagnosis": [ + { + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2" + } + } + ], + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:4d516bf3-2917-46e6-87be-51264b421e30" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:9f541dc0-650b-4558-a996-6e7368c0ec72" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" + } + ] + }, + { + "sequence": 2, + "diagnosisLinkId": [ + 1 + ] + }, + { + "sequence": 3, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "252160004" + } + ] + }, + "net": { + "value": 4119.91, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "procedureLinkId": [ + 2 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "169230002" + } + ] + }, + "net": { + "value": 14013.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:4f7a5693-ac45-4395-9af8-1c63157fec83", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "4f7a5693-ac45-4395-9af8-1c63157fec83", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "80d606df-cf79-487f-8e24-ca7b42363cee" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-08-13T17:09:16-07:00", + "end": "2020-08-13T17:09:16-07:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "diagnosis": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", + "code": "Y", + "display": "Diagnosis present at time of admission" + } + } + ], + "sequence": 1, + "diagnosisReference": { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2" + }, + "type": [ + { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", + "code": "principal" + } + ] + } + ] + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "252160004" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 823.982, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 3295.928, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 4119.91, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 4119.91, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "169230002" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2802.7000000000003, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 11210.800000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 14013.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 14013.50, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 14506.728000000001, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025", + "resource": { + "resourceType": "Encounter", + "id": "3c298587-9c8d-49c3-bde7-7d1e77e12025", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "270427003", + "display": "Patient-initiated encounter" + } + ], + "text": "Patient-initiated encounter" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-08-20T16:24:16-07:00", + "end": "2019-08-20T17:09:16-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04", + "resource": { + "resourceType": "Procedure", + "id": "1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "10383002", + "display": "Counseling for termination of pregnancy" + } + ], + "text": "Counseling for termination of pregnancy" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" + }, + "performedPeriod": { + "start": "2019-08-20T16:24:16-07:00", + "end": "2019-08-20T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:60a85be9-d4a5-42c3-91ed-ee279f501f4a", + "resource": { + "resourceType": "Procedure", + "id": "60a85be9-d4a5-42c3-91ed-ee279f501f4a", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "714812005", + "display": "Induced termination of pregnancy" + } + ], + "text": "Induced termination of pregnancy" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" + }, + "performedPeriod": { + "start": "2019-08-20T16:24:16-07:00", + "end": "2019-08-20T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:edb4ac5a-d52f-4339-85f0-36719a130c3f", + "resource": { + "resourceType": "Claim", + "id": "edb4ac5a-d52f-4339-85f0-36719a130c3f", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-20T16:24:16-07:00", + "end": "2019-08-20T17:09:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:60a85be9-d4a5-42c3-91ed-ee279f501f4a" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "10383002" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 2 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "714812005" + } + ] + }, + "net": { + "value": 11765.34, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:31eb2daf-cb8f-4278-bf01-a9d1e67efed7", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "31eb2daf-cb8f-4278-bf01-a9d1e67efed7", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "edb4ac5a-d52f-4339-85f0-36719a130c3f" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-08-20T17:09:16-07:00", + "end": "2020-08-20T17:09:16-07:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "10383002" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "714812005" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 2353.068, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 9412.272, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 11765.34, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 11765.34, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 9825.592, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03", + "resource": { + "resourceType": "Encounter", + "id": "a7de75eb-43a6-4482-957f-4e9c2b82aa03", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "424619006", + "display": "Prenatal visit" + } + ], + "text": "Prenatal visit" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T17:24:16-07:00" + }, + "reason": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "72892002", + "display": "Normal pregnancy" + } + ] + } + ], + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:0669467d-c7a0-478e-a556-1bdc1674a65c", + "resource": { + "resourceType": "Procedure", + "id": "0669467d-c7a0-478e-a556-1bdc1674a65c", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "386394001", + "display": "Pregnancy termination care" + } + ], + "text": "Pregnancy termination care" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" + }, + "performedPeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:5b87d1f8-e02f-4d3e-9f8d-faf9c4417962", + "resource": { + "resourceType": "Procedure", + "id": "5b87d1f8-e02f-4d3e-9f8d-faf9c4417962", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "5880005", + "display": "Physical exam following abortion" + } + ], + "text": "Physical exam following abortion" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" + }, + "performedPeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:8a47b48c-e479-4d1c-b048-477f419db4f3", + "resource": { + "resourceType": "Procedure", + "id": "8a47b48c-e479-4d1c-b048-477f419db4f3", + "status": "completed", + "code": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "171207006", + "display": "Depression screening" + } + ], + "text": "Depression screening" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" + }, + "performedPeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "reasonReference": [ + { + "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", + "display": "Normal pregnancy" + } + ] + }, + "request": { + "method": "POST", + "url": "Procedure" + } + }, + { + "fullUrl": "urn:uuid:30af96fa-867b-4217-9413-fda59bf8f161", + "resource": { + "resourceType": "Claim", + "id": "30af96fa-867b-4217-9413-fda59bf8f161", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T17:24:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "procedure": [ + { + "sequence": 1, + "procedureReference": { + "reference": "urn:uuid:0669467d-c7a0-478e-a556-1bdc1674a65c" + } + }, + { + "sequence": 2, + "procedureReference": { + "reference": "urn:uuid:5b87d1f8-e02f-4d3e-9f8d-faf9c4417962" + } + }, + { + "sequence": 3, + "procedureReference": { + "reference": "urn:uuid:8a47b48c-e479-4d1c-b048-477f419db4f3" + } + } + ], + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" + } + ] + }, + { + "sequence": 2, + "procedureLinkId": [ + 1 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "386394001" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 3, + "procedureLinkId": [ + 2 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "5880005" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "sequence": 4, + "procedureLinkId": [ + 3 + ], + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "171207006" + } + ] + }, + "net": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:7eeab9d4-aba7-4271-99de-4af8827684bd", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "7eeab9d4-aba7-4271-99de-4af8827684bd", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "30af96fa-867b-4217-9413-fda59bf8f161" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-08-27T17:24:16-07:00", + "end": "2020-08-27T17:24:16-07:00" + }, + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + } + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 2, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "386394001" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 3, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "5880005" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + }, + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 4, + "service": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", + "valueCoding": { + "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", + "code": "0624", + "display": "Dummy" + } + } + ], + "coding": [ + { + "system": "http://snomed.info/sct", + "version": "v1", + "code": "171207006" + } + ] + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "adjudication": [ + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", + "display": "Line Beneficiary Coinsurance Amount" + } + ] + }, + "amount": { + "value": 103.33, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", + "display": "Line Provider Payment Amount" + } + ] + }, + "amount": { + "value": 413.32, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", + "display": "Line Submitted Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", + "display": "Line Allowed Charge Amount" + } + ] + }, + "amount": { + "value": 516.65, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", + "display": "Line Beneficiary Part B Deductible Amount" + } + ] + }, + "amount": { + "value": 0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", + "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", + "display": "Line Processing Indicator Code" + } + ] + } + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 1239.96, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155", + "resource": { + "resourceType": "Encounter", + "id": "51eef314-d8e7-4403-abb5-b8072ed4c155", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "698314001", + "display": "Consultation for treatment" + } + ], + "text": "Consultation for treatment" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + } + ], + "period": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "serviceProvider": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:6b4ab930-b750-4f10-a0e1-df0c554105a9", + "resource": { + "resourceType": "MedicationRequest", + "id": "6b4ab930-b750-4f10-a0e1-df0c554105a9", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "748879", + "display": "Levora 0.15/30 28 Day Pack" + } + ], + "text": "Levora 0.15/30 28 Day Pack" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" + }, + "authoredOn": "2019-08-27T16:24:16-07:00", + "requester": { + "agent": { + "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" + }, + "onBehalfOf": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + } + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:fb9d7e1e-9e63-48ba-9953-a4baab04127e", + "resource": { + "resourceType": "Claim", + "id": "fb9d7e1e-9e63-48ba-9953-a4baab04127e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "prescription": { + "reference": "urn:uuid:6b4ab930-b750-4f10-a0e1-df0c554105a9" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" + } + ] + } + ], + "total": { + "value": 29.37, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:20b108b0-e536-4925-b510-239dc772019e", + "resource": { + "resourceType": "Claim", + "id": "20b108b0-e536-4925-b510-239dc772019e", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "organization": { + "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:255419e8-802f-4fb1-8fd2-a016e9b98a67", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "255419e8-802f-4fb1-8fd2-a016e9b98a67", + "contained": [ + { + "resourceType": "ReferralRequest", + "id": "1", + "status": "completed", + "intent": "order", + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "requester": { + "agent": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + }, + "recipient": [ + { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + } + ] + }, + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "20b108b0-e536-4925-b510-239dc772019e" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-08-27T16:39:16-07:00", + "end": "2020-08-27T16:39:16-07:00" + }, + "created": "2019-08-27T16:39:16-07:00", + "provider": { + "identifier": { + "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" + } + }, + "organization": { + "identifier": { + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + }, + "referral": { + "reference": "#1" + }, + "claim": { + "reference": "urn:uuid:20b108b0-e536-4925-b510-239dc772019e" + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "sequence": 1, + "category": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", + "code": "1", + "display": "Medical care" + } + ] + }, + "servicedPeriod": { + "start": "2019-08-27T16:24:16-07:00", + "end": "2019-08-27T16:39:16-07:00" + }, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "21", + "display": "Inpatient Hospital" + } + ] + }, + "encounter": [ + { + "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" + } + ] + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + }, + { + "fullUrl": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b", + "resource": { + "resourceType": "Encounter", + "id": "5c33c120-05b1-44c0-9156-ca9b8610d22b", + "status": "finished", + "class": { + "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", + "code": "AMB" + }, + "type": [ + { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "162673000", + "display": "General examination of patient (procedure)" + } + ], + "text": "General examination of patient (procedure)" + } + ], + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "participant": [ + { + "individual": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + } + } + ], + "period": { + "start": "2019-12-10T15:24:16-08:00", + "end": "2019-12-10T15:39:16-08:00" + }, + "serviceProvider": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "request": { + "method": "POST", + "url": "Encounter" + } + }, + { + "fullUrl": "urn:uuid:5e921122-3124-49bd-adec-481c0822393c", + "resource": { + "resourceType": "Observation", + "id": "5e921122-3124-49bd-adec-481c0822393c", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8302-2", + "display": "Body Height" + } + ], + "text": "Body Height" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 160.5, + "unit": "cm", + "system": "http://unitsofmeasure.org", + "code": "cm" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:bb03f471-0162-491c-bba0-f92a4427b6b9", + "resource": { + "resourceType": "Observation", + "id": "bb03f471-0162-491c-bba0-f92a4427b6b9", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72514-3", + "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + } + ], + "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 2, + "unit": "{score}", + "system": "http://unitsofmeasure.org", + "code": "{score}" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:9c4305d5-c593-47b2-8018-a433a33ca0e1", + "resource": { + "resourceType": "Observation", + "id": "9c4305d5-c593-47b2-8018-a433a33ca0e1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "29463-7", + "display": "Body Weight" + } + ], + "text": "Body Weight" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 57.100, + "unit": "kg", + "system": "http://unitsofmeasure.org", + "code": "kg" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b1a92a81-99a1-4a1e-8d61-ff1d9873f5e7", + "resource": { + "resourceType": "Observation", + "id": "b1a92a81-99a1-4a1e-8d61-ff1d9873f5e7", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "39156-5", + "display": "Body Mass Index" + } + ], + "text": "Body Mass Index" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 22.170, + "unit": "kg/m2", + "system": "http://unitsofmeasure.org", + "code": "kg/m2" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:48d08279-c0f6-45fe-a213-fcf384be3dc2", + "resource": { + "resourceType": "Observation", + "id": "48d08279-c0f6-45fe-a213-fcf384be3dc2", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "85354-9", + "display": "Blood Pressure" + } + ], + "text": "Blood Pressure" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "component": [ + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8462-4", + "display": "Diastolic Blood Pressure" + } + ], + "text": "Diastolic Blood Pressure" + }, + "valueQuantity": { + "value": 72, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + }, + { + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8480-6", + "display": "Systolic Blood Pressure" + } + ], + "text": "Systolic Blood Pressure" + }, + "valueQuantity": { + "value": 110, + "unit": "mm[Hg]", + "system": "http://unitsofmeasure.org", + "code": "mm[Hg]" + } + } + ] + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:5301d95a-69cc-42d3-8356-afad95c275d1", + "resource": { + "resourceType": "Observation", + "id": "5301d95a-69cc-42d3-8356-afad95c275d1", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "8867-4", + "display": "Heart rate" + } + ], + "text": "Heart rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 91, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:99cedcfc-fff6-4203-9809-7e2587d1f226", + "resource": { + "resourceType": "Observation", + "id": "99cedcfc-fff6-4203-9809-7e2587d1f226", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "vital-signs", + "display": "vital-signs" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "9279-1", + "display": "Respiratory rate" + } + ], + "text": "Respiratory rate" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueQuantity": { + "value": 14, + "unit": "/min", + "system": "http://unitsofmeasure.org", + "code": "/min" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:56f59c4e-1b31-4c26-96fb-258939b6f57e", + "resource": { + "resourceType": "Observation", + "id": "56f59c4e-1b31-4c26-96fb-258939b6f57e", + "status": "final", + "category": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/observation-category", + "code": "survey", + "display": "survey" + } + ] + } + ], + "code": { + "coding": [ + { + "system": "http://loinc.org", + "code": "72166-2", + "display": "Tobacco smoking status NHIS" + } + ], + "text": "Tobacco smoking status NHIS" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "effectiveDateTime": "2019-12-10T15:24:16-08:00", + "issued": "2019-12-10T15:24:16.644-08:00", + "valueCodeableConcept": { + "coding": [ + { + "system": "http://snomed.info/sct", + "code": "266919005", + "display": "Never smoker" + } + ], + "text": "Never smoker" + } + }, + "request": { + "method": "POST", + "url": "Observation" + } + }, + { + "fullUrl": "urn:uuid:b01481fb-444c-4da9-a31a-d6bdbf57b34a", + "resource": { + "resourceType": "MedicationRequest", + "id": "b01481fb-444c-4da9-a31a-d6bdbf57b34a", + "status": "active", + "intent": "order", + "medicationCodeableConcept": { + "coding": [ + { + "system": "http://www.nlm.nih.gov/research/umls/rxnorm", + "code": "746030", + "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + } + ], + "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" + }, + "subject": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "context": { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + }, + "authoredOn": "2019-12-10T15:24:16-08:00", + "requester": { + "agent": { + "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" + }, + "onBehalfOf": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "reasonReference": [ + { + "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" + } + ] + }, + "request": { + "method": "POST", + "url": "MedicationRequest" + } + }, + { + "fullUrl": "urn:uuid:39525a20-1c93-4991-88ee-657bb411d983", + "resource": { + "resourceType": "Claim", + "id": "39525a20-1c93-4991-88ee-657bb411d983", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-12-10T15:24:16-08:00", + "end": "2019-12-10T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "prescription": { + "reference": "urn:uuid:b01481fb-444c-4da9-a31a-d6bdbf57b34a" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + } + ] + } + ], + "total": { + "value": 263.49, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:65dda29e-1896-4adb-8c7d-c7d45a1590f3", + "resource": { + "resourceType": "Claim", + "id": "65dda29e-1896-4adb-8c7d-c7d45a1590f3", + "status": "active", + "use": "complete", + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "start": "2019-12-10T15:24:16-08:00", + "end": "2019-12-10T15:39:16-08:00" + }, + "organization": { + "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" + }, + "item": [ + { + "sequence": 1, + "encounter": [ + { + "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" + } + ] + } + ], + "total": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + "request": { + "method": "POST", + "url": "Claim" + } + }, + { + "fullUrl": "urn:uuid:5ae92d6e-1b87-4356-8957-93d63a82e409", + "resource": { + "resourceType": "ExplanationOfBenefit", + "id": "5ae92d6e-1b87-4356-8957-93d63a82e409", + "meta": { + "profile": [ + "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" + ] + }, + "contained": [ + { + "resourceType": "Coverage", + "id": "coverage", + "type": { + "text": "Blue Cross Blue Shield" + } + } + ], + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", + "code": "N", + "display": "All other reasons for non-payment" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", + "valueMoney": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + }, + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", + "valueIdentifier": { + "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", + "value": "002000" + } + } + ], + "identifier": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/clm_id", + "value": "65dda29e-1896-4adb-8c7d-c7d45a1590f3" + }, + { + "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", + "value": "99999999999" + } + ], + "status": "active", + "type": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", + "code": "71", + "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" + }, + { + "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", + "code": "CARRIER", + "display": "EOB Type" + }, + { + "system": "http://hl7.org/fhir/ex-claimtype", + "code": "professional", + "display": "Claim Type" + }, + { + "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", + "code": "O", + "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" + } + ] + }, + "patient": { + "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" + }, + "billablePeriod": { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", + "valueCoding": { + "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", + "code": "3", + "display": "Final Bill" + } + } + ], + "start": "2019-12-10T15:39:16-08:00", + "end": "2020-12-10T15:39:16-08:00" + }, + "provider": { + "identifier": { + "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" + } + }, + "organization": { + "identifier": { + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + }, + "careTeam": [ + { + "sequence": 1, + "provider": { + "identifier": { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "99999999" + } + }, + "role": { + "coding": [ + { + "system": "http://hl7.org/fhir/claimcareteamrole", + "code": "primary", + "display": "Primary Care Practitioner" + } + ] + } + } + ], + "insurance": { + "coverage": { + "reference": "#coverage" + } + }, + "item": [ + { + "extension": [ + { + "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", + "valueQuantity": { + "value": 0 + } + } + ], + "sequence": 1, + "locationCodeableConcept": { + "coding": [ + { + "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", + "code": "22", + "display": "Outpatient Hospital" + } + ] + } + } + ], + "totalCost": { + "value": 129.16, + "system": "urn:iso:std:iso:4217", + "code": "USD" + }, + "payment": { + "amount": { + "value": 0.0, + "system": "urn:iso:std:iso:4217", + "code": "USD" + } + } + }, + "request": { + "method": "POST", + "url": "ExplanationOfBenefit" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json new file mode 100644 index 000000000000..02eefcfa77eb --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json @@ -0,0 +1,880 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", + "resource": { + "resourceType": "Organization", + "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 11 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "MOUNT AUBURN HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "6174923500" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", + "resource": { + "resourceType": "Organization", + "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 9 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "CAMBRIDGE HEALTH ALLIANCE", + "telecom": [ + { + "system": "phone", + "value": "6176652300" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.375967 + }, + { + "url": "longitude", + "valueDecimal": -71.118275 + } + ] + } + ], + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03", + "resource": { + "resourceType": "Organization", + "id": "5844ad77-f653-3c2b-b7dd-e97576ab3b03", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 46 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "BAYSTATE WING HOSPITAL AND MEDICAL CENTERS", + "telecom": [ + { + "system": "phone", + "value": "4132837651" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.187794 + }, + { + "url": "longitude", + "valueDecimal": -72.30846899999997 + } + ] + } + ], + "line": [ + "40 WRIGHT STREET" + ], + "city": "PALMER", + "state": "MA", + "postalCode": "01069", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", + "resource": { + "resourceType": "Organization", + "id": "c44f361c-2efb-3050-8f97-0354a12e2920", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "c44f361c-2efb-3050-8f97-0354a12e2920" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", + "telecom": [ + { + "system": "phone", + "value": "5089417000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", + "resource": { + "resourceType": "Organization", + "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 8 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 0 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 0 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "GOOD SAMARITAN MEDICAL CENTER", + "telecom": [ + { + "system": "phone", + "value": "5084273000" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.082543 + }, + { + "url": "longitude", + "valueDecimal": -71.024638 + } + ] + } + ], + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245", + "resource": { + "resourceType": "Organization", + "id": "9bd6ea26-d344-3409-aea7-2e925fd0d245", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 17 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 9 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 8 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP32892", + "telecom": [ + { + "system": "phone", + "value": "617-744-6527" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.369451 + }, + { + "url": "longitude", + "valueDecimal": -71.177925 + } + ] + } + ], + "line": [ + "158 A BELMONT ST" + ], + "city": "WATERTOWN", + "state": "MA", + "postalCode": "02472-2912", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5", + "resource": { + "resourceType": "Organization", + "id": "97066a41-4fd5-3d4e-b382-a978fb20a5d5", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 15 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 11 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 2 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 1 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP33234", + "telecom": [ + { + "system": "phone", + "value": "781-829-9300" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.122956 + }, + { + "url": "longitude", + "valueDecimal": -70.85631 + } + ] + } + ], + "line": [ + "198 COLUMBIA RD" + ], + "city": "HANOVER", + "state": "MA", + "postalCode": "02339-2380", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5", + "resource": { + "resourceType": "Organization", + "id": "aced4ce4-957a-3d86-b20a-302d12f949a5", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 9 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 7 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 5 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "LONGFELLOW PRIMARY CARE, PC", + "telecom": [ + { + "system": "phone", + "value": "617-492-4545" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.376043 + }, + { + "url": "longitude", + "valueDecimal": -71.11868 + } + ] + } + ], + "line": [ + "625 MOUNT AUBURN ST" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138-4518", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212", + "resource": { + "resourceType": "Organization", + "id": "83284578-12e5-3582-bfe1-7807e3f7a212", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 8 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 8 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 1 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 4 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "83284578-12e5-3582-bfe1-7807e3f7a212" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "PCP297713", + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.423844 + }, + { + "url": "longitude", + "valueDecimal": -71.10923100000001 + } + ] + } + ], + "line": [ + "0 GOVERNORS AVE" + ], + "city": "MEDFORD", + "state": "MA", + "postalCode": "02155-3084", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", + "resource": { + "resourceType": "Organization", + "id": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 20 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 9 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 3 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 4 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "SIGNATURE HEALTHCARE MEDICAL GROUP INC", + "telecom": [ + { + "system": "phone", + "value": "781-878-1700" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.119966999999995 + }, + { + "url": "longitude", + "valueDecimal": -70.957211 + } + ] + } + ], + "line": [ + "360 BROCKTON AVE" + ], + "city": "ABINGTON", + "state": "MA", + "postalCode": "02351-2186", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + }, + { + "fullUrl": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4", + "resource": { + "resourceType": "Organization", + "id": "883860d6-0862-3a47-a18f-7256563c53f4", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 34 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", + "valueInteger": 50 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", + "valueInteger": 5 + }, + { + "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", + "valueInteger": 29 + } + ], + "identifier": [ + { + "system": "https://github.com/synthetichealth/synthea", + "value": "883860d6-0862-3a47-a18f-7256563c53f4" + } + ], + "type": [ + { + "coding": [ + { + "system": "http://hl7.org/fhir/organization-type", + "code": "prov", + "display": "Healthcare Provider" + } + ], + "text": "Healthcare Provider" + } + ], + "name": "HARRINGTON PHYSICIAN SERVICES INC", + "telecom": [ + { + "system": "phone", + "value": "413-245-0966" + } + ], + "address": [ + { + "extension": [ + { + "url": "http://hl7.org/fhir/StructureDefinition/geolocation", + "extension": [ + { + "url": "latitude", + "valueDecimal": 42.128176 + }, + { + "url": "longitude", + "valueDecimal": -72.205352 + } + ] + } + ], + "line": [ + "255 E OLD STURBRIDGE RD" + ], + "city": "BRIMFIELD", + "state": "MA", + "postalCode": "01010-9647", + "country": "US" + } + ] + }, + "request": { + "method": "POST", + "url": "Organization" + } + } + ] +} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json new file mode 100644 index 000000000000..32853f75bf0c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json @@ -0,0 +1,523 @@ +{ + "resourceType": "Bundle", + "type": "transaction", + "entry": [ + { + "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", + "resource": { + "resourceType": "Practitioner", + "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 11 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "10" + } + ], + "active": true, + "name": [ + { + "family": "Jenkins714", + "given": [ + "Lara964" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "330 MOUNT AUBURN STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", + "resource": { + "resourceType": "Practitioner", + "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 9 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "40" + } + ], + "active": true, + "name": [ + { + "family": "Gibson10", + "given": [ + "Loretta235" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "1493 CAMBRIDGE STREET" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9", + "resource": { + "resourceType": "Practitioner", + "id": "42f9935b-696e-37c9-8261-2a8c72d02dc9", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 46 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "130" + } + ], + "active": true, + "name": [ + { + "family": "Abernathy524", + "given": [ + "Kirby843" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "40 WRIGHT STREET" + ], + "city": "PALMER", + "state": "MA", + "postalCode": "01069", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", + "resource": { + "resourceType": "Practitioner", + "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 2 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "200" + } + ], + "active": true, + "name": [ + { + "family": "Padberg411", + "given": [ + "Nila48" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "680 CENTER STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02302", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "resource": { + "resourceType": "Practitioner", + "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 8 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "460" + } + ], + "active": true, + "name": [ + { + "family": "Fay398", + "given": [ + "Deneen201" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "235 NORTH PEARL STREET" + ], + "city": "BROCKTON", + "state": "MA", + "postalCode": "02301", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0", + "resource": { + "resourceType": "Practitioner", + "id": "ba314750-701e-3370-ade0-270e178d80e0", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 17 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "22240" + } + ], + "active": true, + "name": [ + { + "family": "Pacocha935", + "given": [ + "Ula130" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "158 A BELMONT ST" + ], + "city": "WATERTOWN", + "state": "MA", + "postalCode": "02472-2912", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553", + "resource": { + "resourceType": "Practitioner", + "id": "f9f09686-21e4-3b27-ba68-9b7313d9a553", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 15 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "22570" + } + ], + "active": true, + "name": [ + { + "family": "Homenick806", + "given": [ + "Evan94" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "198 COLUMBIA RD" + ], + "city": "HANOVER", + "state": "MA", + "postalCode": "02339-2380", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886", + "resource": { + "resourceType": "Practitioner", + "id": "29e6e246-9052-309c-a47f-c2e5bc60c886", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 9 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "24070" + } + ], + "active": true, + "name": [ + { + "family": "Waelchi213", + "given": [ + "Cristopher265" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "625 MOUNT AUBURN ST" + ], + "city": "CAMBRIDGE", + "state": "MA", + "postalCode": "02138-4518", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499", + "resource": { + "resourceType": "Practitioner", + "id": "d726e757-2975-3d6a-bdf9-517546432499", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 8 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "80690" + } + ], + "active": true, + "name": [ + { + "family": "Abernathy524", + "given": [ + "Ulrike696" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "0 GOVERNORS AVE" + ], + "city": "MEDFORD", + "state": "MA", + "postalCode": "02155-3084", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841", + "resource": { + "resourceType": "Practitioner", + "id": "3dde31b6-dbc2-394b-b2af-a3167d1f6841", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 20 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "81770" + } + ], + "active": true, + "name": [ + { + "family": "Ondricka197", + "given": [ + "Leisa54" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "360 BROCKTON AVE" + ], + "city": "ABINGTON", + "state": "MA", + "postalCode": "02351-2186", + "country": "US" + } + ], + "gender": "female" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + }, + { + "fullUrl": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915", + "resource": { + "resourceType": "Practitioner", + "id": "7212c355-6c66-3c09-b527-bb8d33b0a915", + "extension": [ + { + "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", + "valueInteger": 34 + } + ], + "identifier": [ + { + "system": "http://hl7.org/fhir/sid/us-npi", + "value": "85760" + } + ], + "active": true, + "name": [ + { + "family": "Kihn564", + "given": [ + "Hayden835" + ], + "prefix": [ + "Dr." + ] + } + ], + "address": [ + { + "line": [ + "255 E OLD STURBRIDGE RD" + ], + "city": "BRIMFIELD", + "state": "MA", + "postalCode": "01010-9647", + "country": "US" + } + ], + "gender": "male" + }, + "request": { + "method": "POST", + "url": "Practitioner" + } + } + ] +} From b33ed49f75e2b3074211a8d907cab91270828fea Mon Sep 17 00:00:00 2001 From: Niel Markwick Date: Thu, 21 May 2020 03:24:03 +0200 Subject: [PATCH 041/151] [BEAM-9822] Simplify pipeline when batching is disabled. (#11529) * Disable grouping by default when streaming. Grouping adds significant latency and memory use, and when streaming this causes both OOMs and high pipeline latencies. * Simplify pipeline when batching is disabled. When batching is disabled, there is no need for SpannerIO to read the schema, group, sort, batch and write batches, so simplify the pipeline to just write the mutation. * Fix noBatching test --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 166 ++++++++++-------- .../io/gcp/spanner/SpannerIOWriteTest.java | 20 ++- 2 files changed, 109 insertions(+), 77 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 07f227ad250a..94ad154ff7b0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -59,6 +59,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; @@ -81,6 +82,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; @@ -1022,84 +1024,98 @@ public void populateDisplayData(DisplayData.Builder builder) { @Override public SpannerWriteResult expand(PCollection input) { + PCollection> batches; + + if (spec.getBatchSizeBytes() <= 1 + || spec.getMaxNumMutations() <= 1 + || spec.getMaxNumRows() <= 1) { + LOG.info("Batching of mutationGroups is disabled"); + TypeDescriptor> descriptor = + new TypeDescriptor>() {}; + batches = + input.apply(MapElements.into(descriptor).via(element -> ImmutableList.of(element))); + } else { - // First, read the Cloud Spanner schema. - PCollection schemaSeed = - input.getPipeline().apply("Create Seed", Create.of((Void) null)); - if (spec.getSchemaReadySignal() != null) { - // Wait for external signal before reading schema. - schemaSeed = schemaSeed.apply("Wait for schema", Wait.on(spec.getSchemaReadySignal())); + // First, read the Cloud Spanner schema. + PCollection schemaSeed = + input.getPipeline().apply("Create Seed", Create.of((Void) null)); + if (spec.getSchemaReadySignal() != null) { + // Wait for external signal before reading schema. + schemaSeed = schemaSeed.apply("Wait for schema", Wait.on(spec.getSchemaReadySignal())); + } + final PCollectionView schemaView = + schemaSeed + .apply( + "Read information schema", + ParDo.of(new ReadSpannerSchema(spec.getSpannerConfig()))) + .apply("Schema View", View.asSingleton()); + + // Split the mutations into batchable and unbatchable mutations. + // Filter out mutation groups too big to be batched. + PCollectionTuple filteredMutations = + input + .apply( + "RewindowIntoGlobal", + Window.into(new GlobalWindows()) + .triggering(DefaultTrigger.of()) + .discardingFiredPanes()) + .apply( + "Filter Unbatchable Mutations", + ParDo.of( + new BatchableMutationFilterFn( + schemaView, + UNBATCHABLE_MUTATIONS_TAG, + spec.getBatchSizeBytes(), + spec.getMaxNumMutations(), + spec.getMaxNumRows())) + .withSideInputs(schemaView) + .withOutputTags( + BATCHABLE_MUTATIONS_TAG, TupleTagList.of(UNBATCHABLE_MUTATIONS_TAG))); + + // Build a set of Mutation groups from the current bundle, + // sort them by table/key then split into batches. + PCollection> batchedMutations = + filteredMutations + .get(BATCHABLE_MUTATIONS_TAG) + .apply( + "Gather And Sort", + ParDo.of( + new GatherBundleAndSortFn( + spec.getBatchSizeBytes(), + spec.getMaxNumMutations(), + spec.getMaxNumRows(), + // Do not group on streaming unless explicitly set. + spec.getGroupingFactor() + .orElse( + input.isBounded() == IsBounded.BOUNDED + ? DEFAULT_GROUPING_FACTOR + : 1), + schemaView)) + .withSideInputs(schemaView)) + .apply( + "Create Batches", + ParDo.of( + new BatchFn( + spec.getBatchSizeBytes(), + spec.getMaxNumMutations(), + spec.getMaxNumRows(), + schemaView)) + .withSideInputs(schemaView)); + + // Merge the batched and unbatchable mutation PCollections and write to Spanner. + batches = + PCollectionList.of(filteredMutations.get(UNBATCHABLE_MUTATIONS_TAG)) + .and(batchedMutations) + .apply("Merge", Flatten.pCollections()); } - final PCollectionView schemaView = - schemaSeed - .apply( - "Read information schema", - ParDo.of(new ReadSpannerSchema(spec.getSpannerConfig()))) - .apply("Schema View", View.asSingleton()); - - // Split the mutations into batchable and unbatchable mutations. - // Filter out mutation groups too big to be batched. - PCollectionTuple filteredMutations = - input - .apply( - "RewindowIntoGlobal", - Window.into(new GlobalWindows()) - .triggering(DefaultTrigger.of()) - .discardingFiredPanes()) - .apply( - "Filter Unbatchable Mutations", - ParDo.of( - new BatchableMutationFilterFn( - schemaView, - UNBATCHABLE_MUTATIONS_TAG, - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows())) - .withSideInputs(schemaView) - .withOutputTags( - BATCHABLE_MUTATIONS_TAG, TupleTagList.of(UNBATCHABLE_MUTATIONS_TAG))); - - // Build a set of Mutation groups from the current bundle, - // sort them by table/key then split into batches. - PCollection> batchedMutations = - filteredMutations - .get(BATCHABLE_MUTATIONS_TAG) - .apply( - "Gather And Sort", - ParDo.of( - new GatherBundleAndSortFn( - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows(), - // Do not group on streaming unless explicitly set. - spec.getGroupingFactor() - .orElse( - input.isBounded() == IsBounded.BOUNDED - ? DEFAULT_GROUPING_FACTOR - : 1), - schemaView)) - .withSideInputs(schemaView)) - .apply( - "Create Batches", - ParDo.of( - new BatchFn( - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows(), - schemaView)) - .withSideInputs(schemaView)); - - // Merge the batchable and unbatchable mutation PCollections and write to Spanner. + PCollectionTuple result = - PCollectionList.of(filteredMutations.get(UNBATCHABLE_MUTATIONS_TAG)) - .and(batchedMutations) - .apply("Merge", Flatten.pCollections()) - .apply( - "Write mutations to Spanner", - ParDo.of( - new WriteToSpannerFn( - spec.getSpannerConfig(), spec.getFailureMode(), FAILED_MUTATIONS_TAG)) - .withOutputTags(MAIN_OUT_TAG, TupleTagList.of(FAILED_MUTATIONS_TAG))); + batches.apply( + "Write batches to Spanner", + ParDo.of( + new WriteToSpannerFn( + spec.getSpannerConfig(), spec.getFailureMode(), FAILED_MUTATIONS_TAG)) + .withOutputTags(MAIN_OUT_TAG, TupleTagList.of(FAILED_MUTATIONS_TAG))); return new SpannerWriteResult( input.getPipeline(), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java index ffcbaefff9a5..816556cf1bc3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java @@ -263,18 +263,34 @@ private void verifyBatches(Iterable... batches) { @Test public void noBatching() throws Exception { + + // This test uses a different mock/fake because it explicitly does not want to populate the + // Spanner schema. + FakeServiceFactory fakeServiceFactory = new FakeServiceFactory(); + ReadOnlyTransaction tx = mock(ReadOnlyTransaction.class); + when(fakeServiceFactory.mockDatabaseClient().readOnlyTransaction()).thenReturn(tx); + + // Capture batches sent to writeAtLeastOnce. + when(fakeServiceFactory.mockDatabaseClient().writeAtLeastOnce(mutationBatchesCaptor.capture())) + .thenReturn(null); + PCollection mutations = pipeline.apply(Create.of(g(m(1L)), g(m(2L)))); mutations.apply( SpannerIO.write() .withProjectId("test-project") .withInstanceId("test-instance") .withDatabaseId("test-database") - .withServiceFactory(serviceFactory) + .withServiceFactory(fakeServiceFactory) .withBatchSizeBytes(1) .grouped()); pipeline.run(); - verifyBatches(batch(m(1L)), batch(m(2L))); + verify(fakeServiceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(mutationsInNoOrder(batch(m(1L)))); + verify(fakeServiceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(mutationsInNoOrder(batch(m(2L)))); + // If no batching then the DB schema is never read. + verify(tx, never()).executeQuery(any()); } @Test From 9cf6f5f3b0beb27b53fac6bd895114fef4b0fe6d Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 20 May 2020 18:52:31 -0700 Subject: [PATCH 042/151] Moving to 2.23.0-SNAPSHOT on master branch. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- gradle.properties | 4 ++-- sdks/python/apache_beam/version.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index eaf79fbdd696..5757ef79af81 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -311,7 +311,7 @@ class BeamModulePlugin implements Plugin { // Automatically use the official release version if we are performing a release // otherwise append '-SNAPSHOT' - project.version = '2.22.0' + project.version = '2.23.0' if (!isRelease(project)) { project.version += '-SNAPSHOT' } diff --git a/gradle.properties b/gradle.properties index e29a95f52554..22f2c251bd7e 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,8 +23,8 @@ offlineRepositoryRoot=offline-repository signing.gnupg.executable=gpg signing.gnupg.useLegacyGpg=true -version=2.22.0-SNAPSHOT -sdk_version=2.22.0.dev +version=2.23.0-SNAPSHOT +sdk_version=2.23.0.dev javaVersion=1.8 diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py index 154e15b9a6f5..964e46428565 100644 --- a/sdks/python/apache_beam/version.py +++ b/sdks/python/apache_beam/version.py @@ -17,4 +17,4 @@ """Apache Beam SDK version information and utilities.""" -__version__ = '2.22.0.dev' +__version__ = '2.23.0.dev' From ffd74b0c07bd9a89ffe021cfcb15cc5fbf40d9f9 Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Wed, 20 May 2020 20:13:13 -0700 Subject: [PATCH 043/151] [BEAM-10028] Add support for the state backed iterable coder to the Java SDK harness. (#11746) * [BEAM-10028] Add support for the state backed iterable coder to the Java SDK harness. This required supporting a translation context through CoderTranslator to give access to the BeamFnStateClient and current process bundle instruction id. * fixup! Address PR comments --- .../AvroGenericCoderTranslator.java | 4 +- .../core/construction/CoderTranslation.java | 35 ++- .../core/construction/CoderTranslator.java | 8 +- .../core/construction/CoderTranslators.java | 9 +- .../construction/ModelCoderRegistrar.java | 11 +- .../core/construction/ModelCoders.java | 12 +- .../construction/RehydratedComponents.java | 4 +- .../construction/CoderTranslationTest.java | 5 +- .../core/construction/CommonCoderTest.java | 3 +- .../construction/ParDoTranslationTest.java | 4 +- .../construction/PipelineTranslationTest.java | 5 +- .../beam/runners/dataflow/DataflowRunner.java | 4 +- .../dataflow/worker/FnApiWindowMappingFn.java | 9 +- .../beam/sdk/coders/IterableLikeCoder.java | 22 +- .../BufferedElementCountingOutputStream.java | 14 +- ...fferedElementCountingOutputStreamTest.java | 16 +- .../expansion/service/ExpansionService.java | 3 +- .../beam/fn/harness/BeamFnDataReadRunner.java | 18 +- .../fn/harness/BeamFnDataWriteRunner.java | 26 +- .../beam/fn/harness/state/BagUserState.java | 2 +- .../fn/harness/state/MultimapSideInput.java | 6 +- .../fn/harness/state/StateBackedIterable.java | 224 ++++++++++++++++++ .../harness/state/StateFetchingIterators.java | 2 +- .../fn/harness/BeamFnDataReadRunnerTest.java | 1 + .../fn/harness/BeamFnDataWriteRunnerTest.java | 3 +- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 7 +- .../state/StateBackedIterableTest.java | 184 ++++++++++++++ 27 files changed, 600 insertions(+), 41 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java index 564a8d3db3cc..53a566c1b52c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/AvroGenericCoderTranslator.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.List; import org.apache.avro.Schema; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.AvroGenericCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; @@ -37,7 +38,8 @@ public byte[] getPayload(AvroGenericCoder from) { } @Override - public AvroGenericCoder fromComponents(List> components, byte[] payload) { + public AvroGenericCoder fromComponents( + List> components, byte[] payload, TranslationContext context) { Schema schema = new Schema.Parser().parse(new String(payload, Charsets.UTF_8)); return AvroGenericCoder.of(schema); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java index 86f017814d53..6f6c59b20df9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java @@ -36,6 +36,22 @@ /** Converts to and from Beam Runner API representations of {@link Coder Coders}. */ public class CoderTranslation { + + /** + * Pass through additional parameters beyond the components and payload to be able to translate + * specific coders. + * + *

    Portability state API backed coders is an example of such a coder translator requiring + * additional parameters. + */ + public interface TranslationContext { + /** The default translation context containing no additional parameters. */ + TranslationContext DEFAULT = new DefaultTranslationContext(); + } + + /** A convenient class representing a default context containing no additional parameters. */ + private static class DefaultTranslationContext implements TranslationContext {} + // This URN says that the coder is just a UDF blob this SDK understands // TODO: standardize such things public static final String JAVA_SERIALIZED_CODER_URN = "beam:coders:javasdk:0.1"; @@ -115,21 +131,29 @@ private static RunnerApi.Coder toCustomCoder(Coder coder) throws IOException .build(); } - public static Coder fromProto(RunnerApi.Coder protoCoder, RehydratedComponents components) + public static Coder fromProto( + RunnerApi.Coder protoCoder, RehydratedComponents components, TranslationContext context) throws IOException { String coderSpecUrn = protoCoder.getSpec().getUrn(); if (coderSpecUrn.equals(JAVA_SERIALIZED_CODER_URN)) { return fromCustomCoder(protoCoder); } - return fromKnownCoder(protoCoder, components); + return fromKnownCoder(protoCoder, components, context); } - private static Coder fromKnownCoder(RunnerApi.Coder coder, RehydratedComponents components) + private static Coder fromKnownCoder( + RunnerApi.Coder coder, RehydratedComponents components, TranslationContext context) throws IOException { String coderUrn = coder.getSpec().getUrn(); List> coderComponents = new ArrayList<>(); for (String componentId : coder.getComponentCoderIdsList()) { - Coder innerCoder = components.getCoder(componentId); + // Only store coders in RehydratedComponents as long as we are not using a custom + // translation context. + Coder innerCoder = + context == TranslationContext.DEFAULT + ? components.getCoder(componentId) + : fromProto( + components.getComponents().getCodersOrThrow(componentId), components, context); coderComponents.add(innerCoder); } Class coderType = KNOWN_CODER_URNS.inverse().get(coderUrn); @@ -139,7 +163,8 @@ private static Coder fromKnownCoder(RunnerApi.Coder coder, RehydratedComponen "Unknown Coder URN %s. Known URNs: %s", coderUrn, KNOWN_CODER_URNS.values()); - return translator.fromComponents(coderComponents, coder.getSpec().getPayload().toByteArray()); + return translator.fromComponents( + coderComponents, coder.getSpec().getPayload().toByteArray(), context); } private static Coder fromCustomCoder(RunnerApi.Coder protoCoder) throws IOException { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java index 442a445df355..434ac1b0957b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslator.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.core.construction; import java.util.List; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.Coder; /** @@ -41,6 +42,9 @@ default byte[] getPayload(T from) { return new byte[0]; } - /** Create a {@link Coder} from its component {@link Coder coders}. */ - T fromComponents(List> components, byte[] payload); + /** + * Create a {@link Coder} from its component {@link Coder coders} using the specified translation + * context. + */ + T fromComponents(List> components, byte[] payload, TranslationContext context); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java index ae7e9dc4b425..c4f45023d912 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -139,7 +140,7 @@ public byte[] getPayload(WindowedValue.ParamWindowedValueCoder from) { @Override public WindowedValue.ParamWindowedValueCoder fromComponents( - List> components, byte[] payload) { + List> components, byte[] payload, TranslationContext context) { return WindowedValue.ParamWindowedValueCoder.fromComponents(components, payload); } }; @@ -158,7 +159,8 @@ public byte[] getPayload(RowCoder from) { } @Override - public RowCoder fromComponents(List> components, byte[] payload) { + public RowCoder fromComponents( + List> components, byte[] payload, TranslationContext context) { checkArgument( components.isEmpty(), "Expected empty component list, but received: " + components); Schema schema; @@ -175,7 +177,8 @@ public RowCoder fromComponents(List> components, byte[] payload) { public abstract static class SimpleStructuredCoderTranslator> implements CoderTranslator { @Override - public final T fromComponents(List> components, byte[] payload) { + public final T fromComponents( + List> components, byte[] payload, TranslationContext context) { return fromComponents(components); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java index 8ff90630f396..501a82379506 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java @@ -20,6 +20,7 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.service.AutoService; +import java.util.Collections; import java.util.Map; import java.util.Set; import org.apache.beam.sdk.coders.BooleanCoder; @@ -97,7 +98,15 @@ public class ModelCoderRegistrar implements CoderTranslatorRegistrar { CoderTranslator.class.getSimpleName(), Sets.difference(BEAM_MODEL_CODER_URNS.keySet(), BEAM_MODEL_CODERS.keySet())); checkState( - ModelCoders.urns().equals(BEAM_MODEL_CODER_URNS.values()), + Sets.symmetricDifference( + ModelCoders.urns(), + /** + * The state backed iterable coder implementation is environment specific and hence + * is not part of the coder translation checks as these are meant to be used only + * during pipeline construction. + */ + Collections.singleton(ModelCoders.STATE_BACKED_ITERABLE_CODER_URN)) + .equals(BEAM_MODEL_CODER_URNS.values()), "All Model %ss should have an associated java %s", Coder.class.getSimpleName(), Coder.class.getSimpleName()); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java index 7685b730510b..e1c204748bb4 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java @@ -19,6 +19,7 @@ import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; import java.util.Set; @@ -59,6 +60,14 @@ private ModelCoders() {} public static final String ROW_CODER_URN = getUrn(StandardCoders.Enum.ROW); + public static final String STATE_BACKED_ITERABLE_CODER_URN = + "beam:coder:state_backed_iterable:v1"; + + static { + checkState( + STATE_BACKED_ITERABLE_CODER_URN.equals(getUrn(StandardCoders.Enum.STATE_BACKED_ITERABLE))); + } + private static final Set MODEL_CODER_URNS = ImmutableSet.of( BYTES_CODER_URN, @@ -74,7 +83,8 @@ private ModelCoders() {} WINDOWED_VALUE_CODER_URN, DOUBLE_CODER_URN, ROW_CODER_URN, - PARAM_WINDOWED_VALUE_CODER_URN); + PARAM_WINDOWED_VALUE_CODER_URN, + STATE_BACKED_ITERABLE_CODER_URN); public static Set urns() { return MODEL_CODER_URNS; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java index 11d24042c626..c51508b14bfd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java @@ -27,6 +27,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.values.PCollection; @@ -81,7 +82,8 @@ public class RehydratedComponents { public Coder load(String id) throws Exception { @Nullable RunnerApi.Coder coder = components.getCodersOrDefault(id, null); checkState(coder != null, "No coder with id '%s' in serialized components", id); - return CoderTranslation.fromProto(coder, RehydratedComponents.this); + return CoderTranslation.fromProto( + coder, RehydratedComponents.this, TranslationContext.DEFAULT); } }); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java index e6b45c475d52..8bc677214c86 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java @@ -31,6 +31,7 @@ import org.apache.avro.SchemaBuilder; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.BooleanCoder; @@ -165,7 +166,9 @@ public void toAndFromProto() throws Exception { Components encodedComponents = sdkComponents.toComponents(); Coder decodedCoder = CoderTranslation.fromProto( - coderProto, RehydratedComponents.forComponents(encodedComponents)); + coderProto, + RehydratedComponents.forComponents(encodedComponents), + TranslationContext.DEFAULT); assertThat(decodedCoder, equalTo(coder)); if (KNOWN_CODERS.contains(coder)) { diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java index fe08582b9bbb..7b0d5453f829 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java @@ -49,6 +49,7 @@ import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.RunnerApi.StandardCoders; import org.apache.beam.model.pipeline.v1.SchemaApi; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.ByteCoder; import org.apache.beam.sdk.coders.Coder; @@ -407,7 +408,7 @@ private static Coder instantiateCoder(CommonCoder coder) { checkNotNull( translator, "No translator found for common coder class: " + coderType.getSimpleName()); - return translator.fromComponents(components, coder.getPayload()); + return translator.fromComponents(components, coder.getPayload(), new TranslationContext() {}); } @Test diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java index 113186eafc7f..5c3b388447fa 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java @@ -29,6 +29,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -200,7 +201,8 @@ public void toTransformProto() throws Exception { Coder timerCoder = CoderTranslation.fromProto( components.getCodersOrThrow(timerFamilySpec.getTimerFamilyCoderId()), - rehydratedComponents); + rehydratedComponents, + TranslationContext.DEFAULT); assertEquals( org.apache.beam.runners.core.construction.Timer.Coder.of( VarLongCoder.of(), GlobalWindow.Coder.INSTANCE), diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java index 9cffd69ab3ac..63264a6897f3 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java @@ -29,6 +29,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload; import org.apache.beam.model.pipeline.v1.RunnerApi.Components; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -213,7 +214,9 @@ private static Coder getAccumulatorCoder(AppliedPTransform transform .orElseThrow(() -> new IOException("Transform does not contain an AccumulatorCoder")); Components components = sdkComponents.toComponents(); return CoderTranslation.fromProto( - components.getCodersOrThrow(id), RehydratedComponents.forComponents(components)); + components.getCodersOrThrow(id), + RehydratedComponents.forComponents(components), + TranslationContext.DEFAULT); } private static Optional getCombinePayload( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index a4ea1c952094..457634fe6d23 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -62,6 +62,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.runners.core.construction.DeduplicatedFlattenFactory; import org.apache.beam.runners.core.construction.EmptyFlattenAsCreateFactory; import org.apache.beam.runners.core.construction.Environments; @@ -1531,7 +1532,8 @@ private Coder getCoder() throws IOException { (Coder) CoderTranslation.fromProto( coderSpec.getCoder(), - RehydratedComponents.forComponents(coderSpec.getComponents())); + RehydratedComponents.forComponents(coderSpec.getComponents()), + TranslationContext.DEFAULT); } return coder; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java index 07f032416167..d8889b936741 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java @@ -35,6 +35,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler; @@ -136,11 +137,15 @@ public static CacheKey create(FunctionSpec windowMappingFn, BoundedWindow mainWi outboundCoder = (Coder) CoderTranslation.fromProto( - components.getCodersOrThrow(mainInputWindowCoderId), rehydratedComponents); + components.getCodersOrThrow(mainInputWindowCoderId), + rehydratedComponents, + TranslationContext.DEFAULT); inboundCoder = (Coder) CoderTranslation.fromProto( - components.getCodersOrThrow(sideInputWindowCoderId), rehydratedComponents); + components.getCodersOrThrow(sideInputWindowCoderId), + rehydratedComponents, + TranslationContext.DEFAULT); } catch (IOException e) { throw new IllegalStateException( "Unable to create side input window mapping process bundle specification.", e); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java index fcd744caf67b..a4e9c86992a5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java @@ -66,9 +66,25 @@ public Coder getElemCoder() { /** * Builds an instance of {@code IterableT}, this coder's associated {@link Iterable}-like subtype, * from a list of decoded elements. + * + *

    Override {@link #decodeToIterable(List, long, InputStream)} if you need access to the + * terminator value and the {@link InputStream}. */ protected abstract IterableT decodeToIterable(List decodedElements); + /** + * Builds an instance of {@code IterableT}, this coder's associated {@link Iterable}-like subtype, + * from a list of decoded elements with the {@link InputStream} at the position where this coder + * detected the end of the stream. + */ + protected IterableT decodeToIterable( + List decodedElements, long terminatorValue, InputStream in) throws IOException { + throw new IllegalStateException( + String.format( + "%s does not support non zero terminator values. Received stream with terminator %s.", + iterableName, terminatorValue)); + } + ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. @@ -136,7 +152,11 @@ public IterableT decode(InputStream inStream) throws IOException, CoderException count = VarInt.decodeLong(dataInStream); } } - return decodeToIterable(elements); + if (count == 0) { + return decodeToIterable(elements); + } else { + return decodeToIterable(elements, count, inStream); + } } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java index 15aab64787ad..3470dbfd55bd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java @@ -79,6 +79,7 @@ public class BufferedElementCountingOutputStream extends OutputStream { public static final int DEFAULT_BUFFER_SIZE = 64 * 1024; private final ByteBuffer buffer; private final OutputStream os; + private final long terminatorValue; private boolean finished; private long count; @@ -87,15 +88,20 @@ public class BufferedElementCountingOutputStream extends OutputStream { * manner. */ public BufferedElementCountingOutputStream(OutputStream os) { - this(os, DEFAULT_BUFFER_SIZE); + this(os, DEFAULT_BUFFER_SIZE, 0); + } + + public BufferedElementCountingOutputStream(OutputStream os, long terminatorValue) { + this(os, DEFAULT_BUFFER_SIZE, terminatorValue); } /** * Creates an output stream which encodes the number of elements output to it in a streaming * manner with the given {@code bufferSize}. */ - BufferedElementCountingOutputStream(OutputStream os, int bufferSize) { + BufferedElementCountingOutputStream(OutputStream os, int bufferSize, long terminatorValue) { this.os = os; + this.terminatorValue = terminatorValue; this.finished = false; this.count = 0; ByteBuffer buffer = BUFFER_POOL.poll(); @@ -111,8 +117,8 @@ public void finish() throws IOException { return; } flush(); - // Finish the stream by stating that there are 0 elements that follow. - VarInt.encode(0, os); + // Finish the stream with the terminatorValue. + VarInt.encode(terminatorValue, os); if (!BUFFER_POOL.offer(buffer)) { // The pool is full, we can't store the buffer. We just drop the buffer. } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 205f1afe1e52..0d1f7d89b719 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -41,14 +41,22 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** Tests for {@link BufferedElementCountingOutputStream}. */ -@RunWith(JUnit4.class) +@RunWith(Parameterized.class) public class BufferedElementCountingOutputStreamTest { @Rule public final ExpectedException expectedException = ExpectedException.none(); private static final int BUFFER_SIZE = 8; + @Parameterized.Parameters + public static Iterable data() { + return ImmutableList.builder().add(new Object[] {0L}).add(new Object[] {-1L}).build(); + } + + @Parameterized.Parameter(0) + public long terminatorValue; + @Test public void testEmptyValues() throws Exception { testValues(Collections.emptyList()); @@ -219,6 +227,8 @@ private void verifyValues(List expectedValues, InputStream is) throws Ex } } while (count > 0); + assertEquals(terminatorValue, count); + if (expectedValues.isEmpty()) { assertTrue(values.isEmpty()); } else { @@ -229,7 +239,7 @@ private void verifyValues(List expectedValues, InputStream is) throws Ex private BufferedElementCountingOutputStream createAndWriteValues( List values, OutputStream output) throws Exception { BufferedElementCountingOutputStream os = - new BufferedElementCountingOutputStream(output, BUFFER_SIZE); + new BufferedElementCountingOutputStream(output, BUFFER_SIZE, terminatorValue); for (byte[] value : values) { os.markElementStart(); diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 578e87583438..f5707eaafca0 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -39,6 +39,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.construction.BeamUrns; import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.core.construction.PipelineTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; @@ -203,7 +204,7 @@ private static Coder resolveCoder(List coderUrns) throws Exception { RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(componentsBuilder.build()); - return CoderTranslation.fromProto(coder, rehydratedComponents); + return CoderTranslation.fromProto(coder, rehydratedComponents, TranslationContext.DEFAULT); } private static RunnerApi.Coder buildProto( diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index eedb42c2a05f..3b10b33dcdd9 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -34,6 +34,7 @@ import org.apache.beam.fn.harness.data.PCollectionConsumerRegistry; import org.apache.beam.fn.harness.data.PTransformFunctionRegistry; import org.apache.beam.fn.harness.state.BeamFnStateClient; +import org.apache.beam.fn.harness.state.StateBackedIterable.StateBackedIterableTranslationContext; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitRequest; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitRequest.DesiredSplit; @@ -120,6 +121,7 @@ public BeamFnDataReadRunner createRunnerForPTransform( processBundleInstructionId, coders, beamFnDataClient, + beamFnStateClient, addProgressRequestCallback, consumer); startFunctionRegistry.register(pTransformId, runner::registerInputLocation); @@ -149,6 +151,7 @@ public BeamFnDataReadRunner createRunnerForPTransform( Supplier processBundleInstructionIdSupplier, Map coders, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, Consumer addProgressRequestCallback, FnDataReceiver> consumer) throws IOException { @@ -163,7 +166,20 @@ public BeamFnDataReadRunner createRunnerForPTransform( RehydratedComponents.forComponents(Components.newBuilder().putAllCoders(coders).build()); this.coder = (Coder>) - CoderTranslation.fromProto(coders.get(port.getCoderId()), components); + CoderTranslation.fromProto( + coders.get(port.getCoderId()), + components, + new StateBackedIterableTranslationContext() { + @Override + public BeamFnStateClient getStateClient() { + return beamFnStateClient; + } + + @Override + public Supplier getCurrentInstructionId() { + return processBundleInstructionIdSupplier; + } + }); addProgressRequestCallback.accept( () -> { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index c2570c1c8490..684f39911320 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -30,6 +30,7 @@ import org.apache.beam.fn.harness.data.PCollectionConsumerRegistry; import org.apache.beam.fn.harness.data.PTransformFunctionRegistry; import org.apache.beam.fn.harness.state.BeamFnStateClient; +import org.apache.beam.fn.harness.state.StateBackedIterable.StateBackedIterableTranslationContext; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort; import org.apache.beam.model.pipeline.v1.Endpoints; @@ -99,7 +100,12 @@ public BeamFnDataWriteRunner createRunnerForPTransform( BeamFnDataWriteRunner runner = new BeamFnDataWriteRunner<>( - pTransformId, pTransform, processBundleInstructionId, coders, beamFnDataClient); + pTransformId, + pTransform, + processBundleInstructionId, + coders, + beamFnDataClient, + beamFnStateClient); startFunctionRegistry.register(pTransformId, runner::registerForOutput); pCollectionConsumerRegistry.register( getOnlyElement(pTransform.getInputsMap().values()), @@ -124,7 +130,8 @@ public BeamFnDataWriteRunner createRunnerForPTransform( RunnerApi.PTransform remoteWriteNode, Supplier processBundleInstructionIdSupplier, Map coders, - BeamFnDataClient beamFnDataClientFactory) + BeamFnDataClient beamFnDataClientFactory, + BeamFnStateClient beamFnStateClient) throws IOException { this.pTransformId = pTransformId; RemoteGrpcPort port = RemoteGrpcPortWrite.fromPTransform(remoteWriteNode).getPort(); @@ -136,7 +143,20 @@ public BeamFnDataWriteRunner createRunnerForPTransform( RehydratedComponents.forComponents(Components.newBuilder().putAllCoders(coders).build()); this.coder = (Coder>) - CoderTranslation.fromProto(coders.get(port.getCoderId()), components); + CoderTranslation.fromProto( + coders.get(port.getCoderId()), + components, + new StateBackedIterableTranslationContext() { + @Override + public BeamFnStateClient getStateClient() { + return beamFnStateClient; + } + + @Override + public Supplier getCurrentInstructionId() { + return processBundleInstructionIdSupplier; + } + }); } public void registerForOutput() { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java index 38e202700ed0..cb3c1953aa53 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java @@ -77,7 +77,7 @@ public BagUserState( new DataStreams.DataStreamDecoder( valueCoder, DataStreams.inbound( - StateFetchingIterators.forFirstChunk(beamFnStateClient, request)))); + StateFetchingIterators.readAllStartingFrom(beamFnStateClient, request)))); this.newValues = new ArrayList<>(); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java index d795d44f163f..795561047d1a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java @@ -71,7 +71,8 @@ public Iterable get() { new DataStreams.DataStreamDecoder( keyCoder, DataStreams.inbound( - StateFetchingIterators.forFirstChunk(beamFnStateClient, requestBuilder.build())))); + StateFetchingIterators.readAllStartingFrom( + beamFnStateClient, requestBuilder.build())))); } @Override @@ -97,6 +98,7 @@ public Iterable get(K k) { new DataStreams.DataStreamDecoder( valueCoder, DataStreams.inbound( - StateFetchingIterators.forFirstChunk(beamFnStateClient, requestBuilder.build())))); + StateFetchingIterators.readAllStartingFrom( + beamFnStateClient, requestBuilder.build())))); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java new file mode 100644 index 000000000000..94c8f9a8f614 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java @@ -0,0 +1,224 @@ +/* + * 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.beam.fn.harness.state; + +import static org.apache.beam.runners.core.construction.ModelCoders.STATE_BACKED_ITERABLE_CODER_URN; + +import com.google.auto.service.AutoService; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; +import org.apache.beam.runners.core.construction.CoderTranslator; +import org.apache.beam.runners.core.construction.CoderTranslatorRegistrar; +import org.apache.beam.sdk.coders.IterableLikeCoder; +import org.apache.beam.sdk.fn.stream.DataStreams; +import org.apache.beam.sdk.util.BufferedElementCountingOutputStream; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams; + +/** + * A {@link BeamFnStateClient state} backed iterable which allows for fetching elements over the + * portability state API. See remote references for + * additional details. + * + *

    One must supply a {@link StateBackedIterableTranslationContext} when using {@link + * CoderTranslator#fromComponents} to be able to create a {@link StateBackedIterable.Coder}. + */ +public class StateBackedIterable implements Iterable { + + private final BeamFnStateClient beamFnStateClient; + private final org.apache.beam.sdk.coders.Coder elemCoder; + @VisibleForTesting final StateRequest request; + @VisibleForTesting final List prefix; + + public StateBackedIterable( + BeamFnStateClient beamFnStateClient, + String instructionId, + ByteString runnerKey, + org.apache.beam.sdk.coders.Coder elemCoder, + List prefix) { + this.beamFnStateClient = beamFnStateClient; + this.elemCoder = elemCoder; + + StateRequest.Builder requestBuilder = StateRequest.newBuilder(); + requestBuilder + .setInstructionId(instructionId) + .getStateKeyBuilder() + .getRunnerBuilder() + .setKey(runnerKey); + this.request = requestBuilder.build(); + this.prefix = prefix; + } + + @Override + public Iterator iterator() { + return Iterators.concat( + prefix.iterator(), + new DataStreams.DataStreamDecoder( + elemCoder, + DataStreams.inbound( + StateFetchingIterators.readAllStartingFrom(beamFnStateClient, request)))); + } + + /** + * Decodes an {@link Iterable} that might be backed by state. If the terminator at the end of the + * value stream is {@code -1} then we return a {@link StateBackedIterable} otherwise we return an + * {@link Iterable}. + */ + public static class Coder extends IterableLikeCoder> { + + private final BeamFnStateClient beamFnStateClient; + private final Supplier instructionId; + + public Coder( + BeamFnStateClient beamFnStateClient, + Supplier instructionId, + org.apache.beam.sdk.coders.Coder elemCoder) { + super(elemCoder, "StateBackedIterable"); + this.beamFnStateClient = beamFnStateClient; + this.instructionId = instructionId; + } + + @Override + protected Iterable decodeToIterable(List decodedElements) { + return decodedElements; + } + + @Override + protected Iterable decodeToIterable( + List decodedElements, long terminatorValue, InputStream in) throws IOException { + if (terminatorValue == -1L) { + long tokenLength = VarInt.decodeLong(in); + ByteString token = ByteString.readFrom(ByteStreams.limit(in, tokenLength)); + return new StateBackedIterable<>( + beamFnStateClient, instructionId.get(), token, getElemCoder(), decodedElements); + } else { + throw new IllegalStateException( + String.format( + "StateBackedIterable expected terminator of 0 or -1 but received %s.", + terminatorValue)); + } + } + + @Override + public void encode(Iterable iterable, OutputStream outStream) throws IOException { + if (!(iterable instanceof StateBackedIterable)) { + super.encode(iterable, outStream); + return; + } + + StateBackedIterable stateBackedIterable = (StateBackedIterable) iterable; + + DataOutputStream dataOutStream = new DataOutputStream(outStream); + // We don't know the size without traversing it so use a fixed size buffer + // and encode as many elements as possible into it before outputting the size followed + // by the elements. + dataOutStream.writeInt(-1); + BufferedElementCountingOutputStream countingOutputStream = + new BufferedElementCountingOutputStream(dataOutStream, -1L); + // Encode only the prefix + for (T elem : stateBackedIterable.prefix) { + countingOutputStream.markElementStart(); + getElemCoder().encode(elem, countingOutputStream); + } + countingOutputStream.finish(); + // Make sure all our output gets pushed to the underlying outStream. + dataOutStream.flush(); + + // Append 'len(token) token' after the -1 stream terminator. + VarInt.encode( + stateBackedIterable.request.getStateKey().getRunner().getKey().size(), outStream); + stateBackedIterable.request.getStateKey().getRunner().getKey().writeTo(outStream); + } + } + + /** Additional parameters required by the {@link StateBackedIterable.Coder}. */ + public interface StateBackedIterableTranslationContext extends TranslationContext { + BeamFnStateClient getStateClient(); + + Supplier getCurrentInstructionId(); + } + + /** A {@link CoderTranslatorRegistrar} for {@code beam:coder:state_backed_iterable:v1}. */ + @AutoService(CoderTranslatorRegistrar.class) + public static class Registrar implements CoderTranslatorRegistrar { + + @Override + public Map, String> getCoderURNs() { + return Collections.singletonMap( + StateBackedIterable.Coder.class, STATE_BACKED_ITERABLE_CODER_URN); + } + + @Override + public Map< + Class, + CoderTranslator> + getCoderTranslators() { + return ImmutableMap.of(StateBackedIterable.Coder.class, new Translator()); + } + } + + /** + * A {@link CoderTranslator} for {@code beam:coder:state_backed_iterable:v1}. + * + *

    One must supply a {@link StateBackedIterableTranslationContext} during {@link + * CoderTranslator#fromComponents} to be able to successfully create an instance. + */ + private static class Translator implements CoderTranslator> { + + @Override + public List> getComponents( + StateBackedIterable.Coder from) { + return Collections.>singletonList(from.getElemCoder()); + } + + @Override + public StateBackedIterable.Coder fromComponents( + List> components, + byte[] payload, + TranslationContext context) { + if (context instanceof StateBackedIterableTranslationContext) { + return new StateBackedIterable.Coder<>( + ((StateBackedIterableTranslationContext) context).getStateClient(), + ((StateBackedIterableTranslationContext) context).getCurrentInstructionId(), + Iterables.getOnlyElement(components)); + } else { + throw new IllegalStateException( + String.format( + "Unable to construct coder %s. Expected translation context %s but received %s.", + STATE_BACKED_ITERABLE_CODER_URN, + StateBackedIterableTranslationContext.class.getName(), + context.getClass().getName())); + } + } + } +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java index a7cb38d7b25a..7af431de0659 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java @@ -46,7 +46,7 @@ private StateFetchingIterators() {} * only) chunk of a state stream. This state request will be populated with a continuation * token to request further chunks of the stream if required. */ - public static Iterator forFirstChunk( + public static Iterator readAllStartingFrom( BeamFnStateClient beamFnStateClient, StateRequest stateRequestForFirstChunk) { return new LazyBlockingStateFetchingIterator(beamFnStateClient, stateRequestForFirstChunk); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index 75f3105b0b22..9293d011c35a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -235,6 +235,7 @@ public void testReuseForMultipleBundles() throws Exception { bundleId::get, COMPONENTS.getCodersMap(), mockBeamFnDataClient, + null /* beamFnStateClient */, (PTransformRunnerFactory.ProgressRequestCallback callback) -> { progressCallbacks.add(callback); }, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index 7159801d1eee..a2f4c8f27511 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -213,7 +213,8 @@ public void testReuseForMultipleBundles() throws Exception { RemoteGrpcPortWrite.writeToPort("myWrite", PORT_SPEC).toPTransform(), bundleId::get, COMPONENTS.getCodersMap(), - mockBeamFnDataClient); + mockBeamFnDataClient, + null /* beamFnStateClient */); // Process for bundle id 0 writeRunner.registerForOutput(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 1da967269b32..b5e23685e731 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -59,6 +59,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.Environment; import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.CoderTranslation.TranslationContext; import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.core.construction.PipelineTranslation; @@ -1188,7 +1189,8 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { WindowedValue.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), - rehydratedComponents), + rehydratedComponents, + TranslationContext.DEFAULT), (Coder) CoderTranslation.fromProto( pProto @@ -1199,7 +1201,8 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { .getWindowingStrategiesOrThrow( inputPCollection.getWindowingStrategyId()) .getWindowCoderId()), - rehydratedComponents)); + rehydratedComponents, + TranslationContext.DEFAULT)); String outputPCollectionId = pTransform.getOutputsOrThrow("output"); ImmutableMap stateData = diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java new file mode 100644 index 000000000000..fc6549b294a2 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java @@ -0,0 +1,184 @@ +/* + * 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.beam.fn.harness.state; + +import static org.junit.Assert.assertEquals; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; + +/** Tests for {@link StateBackedIterable}. */ +@RunWith(Enclosed.class) +public class StateBackedIterableTest { + + @RunWith(Parameterized.class) + public static class IterationTest { + @Parameterized.Parameters + public static Iterable data() { + return ImmutableList.builder() + .add(new Object[] {Collections.emptyList(), "emptySuffix", ImmutableList.of()}) + .add(new Object[] {ImmutableList.of("A", "B"), "emptySuffix", ImmutableList.of("A", "B")}) + .add( + new Object[] { + Collections.emptyList(), + "nonEmptySuffix", + ImmutableList.of("C", "D", "E", "F", "G", "H", "I", "J", "K") + }) + .add( + new Object[] { + ImmutableList.of("A", "B"), + "nonEmptySuffix", + ImmutableList.of("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K") + }) + .build(); + } + + @Parameterized.Parameter(0) + public List prefix; + + @Parameterized.Parameter(1) + public String suffixKey; + + @Parameterized.Parameter(2) + public List expected; + + @Test + public void testReiteration() throws Exception { + FakeBeamFnStateClient fakeBeamFnStateClient = + new FakeBeamFnStateClient( + ImmutableMap.of( + key("nonEmptySuffix"), encode("C", "D", "E", "F", "G", "H", "I", "J", "K"), + key("emptySuffix"), encode())); + + StateBackedIterable iterable = + new StateBackedIterable<>( + fakeBeamFnStateClient, + "instruction", + encode(suffixKey), + StringUtf8Coder.of(), + prefix); + + assertEquals(expected, Lists.newArrayList(iterable)); + assertEquals(expected, Lists.newArrayList(iterable)); + assertEquals(expected, Lists.newArrayList(iterable)); + } + + @Test + public void testUsingInterleavedReiteration() throws Exception { + FakeBeamFnStateClient fakeBeamFnStateClient = + new FakeBeamFnStateClient( + ImmutableMap.of( + key("nonEmptySuffix"), encode("C", "D", "E", "F", "G", "H", "I", "J", "K"), + key("emptySuffix"), encode())); + + StateBackedIterable iterable = + new StateBackedIterable<>( + fakeBeamFnStateClient, + "instruction", + encode(suffixKey), + StringUtf8Coder.of(), + prefix); + + List> iterators = new ArrayList<>(); + List> results = new ArrayList<>(); + for (int i = 0; i < 3; ++i) { + iterators.add(iterable.iterator()); + results.add(new ArrayList<>()); + } + + Random random = new Random(42L); + while (!iterators.isEmpty()) { + int current = random.nextInt(iterators.size()); + if (!iterators.get(current).hasNext()) { + iterators.remove(current); + assertEquals(expected, results.remove(current)); + } else { + results.get(current).add(iterators.get(current).next()); + } + } + } + } + + @RunWith(JUnit4.class) + public static class CoderTest { + @Test + public void testDecodeEncodeRegularIterable() throws Exception { + Iterable iterable = FluentIterable.of("A", "B", "C"); + StateBackedIterable.Coder coder = + new StateBackedIterable.Coder<>(null, () -> "instructionId", StringUtf8Coder.of()); + + // We can't rely on CoderProperties since it requires serialization of the coder + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + coder.encode(iterable, baos); + Iterable result = coder.decode(new ByteArrayInputStream(baos.toByteArray())); + + assertEquals(Lists.newArrayList(iterable), Lists.newArrayList(result)); + } + + @Test + public void testEncodeDecodeStateBackedIterable() throws Exception { + StateBackedIterable iterable = + new StateBackedIterable( + null, "instructionId", encode("key"), StringUtf8Coder.of(), Arrays.asList("A", "B")); + StateBackedIterable.Coder coder = + new StateBackedIterable.Coder<>(null, () -> "instructionId", StringUtf8Coder.of()); + + // We can't rely on CoderProperties since it requires serialization of the coder + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + coder.encode(iterable, baos); + StateBackedIterable result = + (StateBackedIterable) coder.decode(new ByteArrayInputStream(baos.toByteArray())); + + // Ensure that when we round trip the iterable we correctly convert it back to a state backed + // iterable + assertEquals(iterable.prefix, result.prefix); + assertEquals(iterable.request, result.request); + } + } + + private static StateKey key(String id) throws IOException { + return StateKey.newBuilder().setRunner(StateKey.Runner.newBuilder().setKey(encode(id))).build(); + } + + private static ByteString encode(String... values) throws IOException { + ByteString.Output out = ByteString.newOutput(); + for (String value : values) { + StringUtf8Coder.of().encode(value, out); + } + return out.toByteString(); + } +} From 22d604b45b2597c9f863a82bf688430f4f2d8679 Mon Sep 17 00:00:00 2001 From: Rehman Date: Thu, 21 May 2020 17:42:36 +0500 Subject: [PATCH 044/151] Update changes.md --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 9aff55608c1d..107e7dc8dff1 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,7 @@ * `--workerCacheMB` flag is supported in Dataflow streaming pipeline ([BEAM-9964](https://issues.apache.org/jira/browse/BEAM-9964)) * `--direct_num_workers=0` is supported for FnApi runner. It will set the number of threads/subprocesses to number of cores of the machine executing the pipeline ([BEAM-9443](https://issues.apache.org/jira/browse/BEAM-9443)). * Python SDK now has experimental support for SqlTransform ([BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)). +* Add OnWindowExpiration method to Stateful DoFn ([BEAM-1589](https://issues.apache.org/jira/browse/BEAM-1589)). ## Breaking Changes From d3dfa30f7b6d1ac1ad693e98b29d09cc0586d6de Mon Sep 17 00:00:00 2001 From: Yichi Zhang Date: Thu, 21 May 2020 09:05:24 -0700 Subject: [PATCH 045/151] comments --- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 77 +++++++++++-------- 1 file changed, 45 insertions(+), 32 deletions(-) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index e763fface9f4..07086aa8015d 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -864,10 +864,11 @@ public void testTimers() throws Exception { processingTimer, timerInGlobalWindow("B", new Instant(1900L), new Instant(2900L))); fakeTimerClient.sendTimer( eventFamilyTimer, - timerInGlobalWindow("B", "event-timer2", new Instant(2000L), new Instant(3000L))); + dynamicTimerInGlobalWindow("B", "event-timer2", new Instant(2000L), new Instant(3000L))); fakeTimerClient.sendTimer( processingFamilyTimer, - timerInGlobalWindow("Y", "processing-timer2", new Instant(2100L), new Instant(3100L))); + dynamicTimerInGlobalWindow( + "Y", "processing-timer2", new Instant(2100L), new Instant(3100L))); assertThat( mainOutputValues, contains( @@ -916,33 +917,45 @@ public void testTimers() throws Exception { assertThat( fakeTimerClient.getTimers(eventFamilyTimer), contains( - timerInGlobalWindow("X", "event-timer1", new Instant(1000L), new Instant(1003L)), - timerInGlobalWindow("Y", "event-timer1", new Instant(1100L), new Instant(1103L)), - timerInGlobalWindow("X", "event-timer1", new Instant(1200L), new Instant(1203L)), - timerInGlobalWindow("Y", "event-timer1", new Instant(1300L), new Instant(1303L)), - timerInGlobalWindow("A", "event-timer1", new Instant(1400L), new Instant(2413L)), - timerInGlobalWindow("B", "event-timer1", new Instant(1500L), new Instant(2513L)), - timerInGlobalWindow("A", "event-timer1", new Instant(1600L), new Instant(2613L)), - timerInGlobalWindow("X", "event-timer1", new Instant(1700L), new Instant(1723L)), - timerInGlobalWindow("C", "event-timer1", new Instant(1800L), new Instant(1823L)), - timerInGlobalWindow("B", "event-timer1", new Instant(1900L), new Instant(1923L)), - timerInGlobalWindow("B", "event-timer1", new Instant(2000L), new Instant(2033L)), - timerInGlobalWindow("Y", "event-timer1", new Instant(2100L), new Instant(2143L)))); + dynamicTimerInGlobalWindow("X", "event-timer1", new Instant(1000L), new Instant(1003L)), + dynamicTimerInGlobalWindow("Y", "event-timer1", new Instant(1100L), new Instant(1103L)), + dynamicTimerInGlobalWindow("X", "event-timer1", new Instant(1200L), new Instant(1203L)), + dynamicTimerInGlobalWindow("Y", "event-timer1", new Instant(1300L), new Instant(1303L)), + dynamicTimerInGlobalWindow("A", "event-timer1", new Instant(1400L), new Instant(2413L)), + dynamicTimerInGlobalWindow("B", "event-timer1", new Instant(1500L), new Instant(2513L)), + dynamicTimerInGlobalWindow("A", "event-timer1", new Instant(1600L), new Instant(2613L)), + dynamicTimerInGlobalWindow("X", "event-timer1", new Instant(1700L), new Instant(1723L)), + dynamicTimerInGlobalWindow("C", "event-timer1", new Instant(1800L), new Instant(1823L)), + dynamicTimerInGlobalWindow("B", "event-timer1", new Instant(1900L), new Instant(1923L)), + dynamicTimerInGlobalWindow("B", "event-timer1", new Instant(2000L), new Instant(2033L)), + dynamicTimerInGlobalWindow( + "Y", "event-timer1", new Instant(2100L), new Instant(2143L)))); assertThat( fakeTimerClient.getTimers(processingFamilyTimer), contains( - timerInGlobalWindow("X", "processing-timer1", new Instant(1000L), new Instant(10004L)), - timerInGlobalWindow("Y", "processing-timer1", new Instant(1100L), new Instant(10004L)), - timerInGlobalWindow("X", "processing-timer1", new Instant(1200L), new Instant(10004L)), - timerInGlobalWindow("Y", "processing-timer1", new Instant(1300L), new Instant(10004L)), - timerInGlobalWindow("A", "processing-timer1", new Instant(1400L), new Instant(10014L)), - timerInGlobalWindow("B", "processing-timer1", new Instant(1500L), new Instant(10014L)), - timerInGlobalWindow("A", "processing-timer1", new Instant(1600L), new Instant(10014L)), - timerInGlobalWindow("X", "processing-timer1", new Instant(1700L), new Instant(10024L)), - timerInGlobalWindow("C", "processing-timer1", new Instant(1800L), new Instant(10024L)), - timerInGlobalWindow("B", "processing-timer1", new Instant(1900L), new Instant(10024L)), - timerInGlobalWindow("B", "processing-timer1", new Instant(2000L), new Instant(10034L)), - timerInGlobalWindow( + dynamicTimerInGlobalWindow( + "X", "processing-timer1", new Instant(1000L), new Instant(10004L)), + dynamicTimerInGlobalWindow( + "Y", "processing-timer1", new Instant(1100L), new Instant(10004L)), + dynamicTimerInGlobalWindow( + "X", "processing-timer1", new Instant(1200L), new Instant(10004L)), + dynamicTimerInGlobalWindow( + "Y", "processing-timer1", new Instant(1300L), new Instant(10004L)), + dynamicTimerInGlobalWindow( + "A", "processing-timer1", new Instant(1400L), new Instant(10014L)), + dynamicTimerInGlobalWindow( + "B", "processing-timer1", new Instant(1500L), new Instant(10014L)), + dynamicTimerInGlobalWindow( + "A", "processing-timer1", new Instant(1600L), new Instant(10014L)), + dynamicTimerInGlobalWindow( + "X", "processing-timer1", new Instant(1700L), new Instant(10024L)), + dynamicTimerInGlobalWindow( + "C", "processing-timer1", new Instant(1800L), new Instant(10024L)), + dynamicTimerInGlobalWindow( + "B", "processing-timer1", new Instant(1900L), new Instant(10024L)), + dynamicTimerInGlobalWindow( + "B", "processing-timer1", new Instant(2000L), new Instant(10034L)), + dynamicTimerInGlobalWindow( "Y", "processing-timer1", new Instant(2100L), new Instant(10044L)))); mainOutputValues.clear(); @@ -979,14 +992,10 @@ public void testTimers() throws Exception { private org.apache.beam.runners.core.construction.Timer timerInGlobalWindow( K userKey, Instant holdTimestamp, Instant fireTimestamp) { - return timerInGlobalWindow(userKey, "", holdTimestamp, fireTimestamp); + return dynamicTimerInGlobalWindow(userKey, "", holdTimestamp, fireTimestamp); } - private WindowedValue valueInWindow(T value, BoundedWindow window) { - return WindowedValue.of(value, window.maxTimestamp(), window, PaneInfo.NO_FIRING); - } - - private org.apache.beam.runners.core.construction.Timer timerInGlobalWindow( + private org.apache.beam.runners.core.construction.Timer dynamicTimerInGlobalWindow( K userKey, String dynamicTimerTag, Instant holdTimestamp, Instant fireTimestamp) { return org.apache.beam.runners.core.construction.Timer.of( userKey, @@ -997,6 +1006,10 @@ private org.apache.beam.runners.core.construction.Timer timerInGlobalWind PaneInfo.NO_FIRING); } + private WindowedValue valueInWindow(T value, BoundedWindow window) { + return WindowedValue.of(value, window.maxTimestamp(), window, PaneInfo.NO_FIRING); + } + private static class TestTimerfulDoFn extends DoFn, String> { @StateId("bag") private final StateSpec> bagStateSpec = StateSpecs.bag(StringUtf8Coder.of()); From 79e84e98f7de076de72302a4bce9086236e1e947 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Thu, 21 May 2020 12:40:06 -0400 Subject: [PATCH 046/151] Include additional instructions for staging repository. --- website/www/site/content/en/contribute/release-guide.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 3da9fbcc3973..230cae418ed8 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -607,7 +607,10 @@ For this step, we recommend you using automation script to create a RC, but you ls -al /opt/apache/beam/third_party_licenses/ | wc -l ``` 1. Publish staging artifacts - 1. Go to the staging repo to close the staging repository on [Apache Nexus](https://repository.apache.org/#stagingRepositories). + 1. Log in to the [Apache Nexus](https://repository.apache.org/#stagingRepositories) website. + 1. Navigate to Build Promotion -> Staging Repositories (in the left sidebar). + 1. Select repository `orgapachebeam-NNNN`. + 1. Click the Close button. 1. When prompted for a description, enter “Apache Beam, version X, release candidate Y”. 1. Review all staged artifacts on https://repository.apache.org/content/repositories/orgapachebeam-NNNN/. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.beam%22) in which artifact name mirrors directory structure, e.g., `beam-sdks-java-io-kafka`. Carefully review any new artifacts. 1. Build and stage python wheels. From b8e582ffaf39918819640c83f4b3fdf9f0190abf Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Thu, 21 May 2020 01:01:24 -0700 Subject: [PATCH 047/151] [BEAM-10052] check hash and avoid duplicated artifacts --- .../runners/dataflow/internal/apiclient.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index 06bb26029aeb..55649a761af7 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -583,6 +583,7 @@ def _stage_resources(self, pipeline, options): raise RuntimeError('The --temp_location option must be specified.') resources = [] + hashs = {} for _, env in sorted(pipeline.components.environments.items(), key=lambda kv: kv[0]): for dep in env.dependencies: @@ -595,7 +596,16 @@ def _stage_resources(self, pipeline, options): role_payload = ( beam_runner_api_pb2.ArtifactStagingToRolePayload.FromString( dep.role_payload)) - resources.append((type_payload.path, role_payload.staged_name)) + if type_payload.sha256 and type_payload.sha256 in hashs: + _LOGGER.info( + 'Found duplicated artifact: %s (%s)', + type_payload.path, + type_payload.sha256) + dep.role_payload = beam_runner_api_pb2.ArtifactStagingToRolePayload( + staged_name=hashs[type_payload.sha256]).SerializeToString() + else: + resources.append((type_payload.path, role_payload.staged_name)) + hashs[type_payload.sha256] = role_payload.staged_name resource_stager = _LegacyDataflowStager(self) staged_resources = resource_stager.stage_job_resources( From 0c3780c438bc1e3c9e314e87e8646e3f4e8972b4 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Thu, 21 May 2020 13:54:42 -0400 Subject: [PATCH 048/151] Clarify the usage of sign_hash_python_wheels.sh. --- website/www/site/content/en/contribute/release-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 230cae418ed8..3037698002f8 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -617,8 +617,8 @@ For this step, we recommend you using automation script to create a RC, but you - There is a wrapper repo [beam-wheels](https://github.com/apache/beam-wheels) to help build python wheels. - If you are interested in how it works, please refer to the [structure section](https://github.com/apache/beam-wheels#structure). - Please follow the [user guide](https://github.com/apache/beam-wheels#user-guide) to build python wheels. - - Once all python wheels have been staged [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/), - please run [./sign_hash_python_wheels.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/sign_hash_python_wheels.sh) to sign and hash python wheels. + - Once all python wheels have been staged to GCS, + please run [./sign_hash_python_wheels.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/sign_hash_python_wheels.sh), which copies the wheels along with signatures and hashes to [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/). 1. Update Beam website ([example](https://github.com/apache/beam/pull/11727)) 1. Update release version in `website/www/site/config.toml`. 1. Add new release in `website/www/site/content/en/get-started/downloads.md`. From 105f2657a730672b70fbb3ec4a2f111247a5cae1 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 21 May 2020 15:03:23 -0700 Subject: [PATCH 049/151] Website build: Remove -u root (#11625) --- website/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/build.gradle b/website/build.gradle index 235cb8426e6d..b5988d8e2b42 100644 --- a/website/build.gradle +++ b/website/build.gradle @@ -119,7 +119,7 @@ task setupDockerContainer(type: Exec) { } // Create the config to point to a GitHub or Colab blob in the repo, e.g. apache/beam/blob/master - commandLine 'docker', 'exec', '-u', 'root', + commandLine 'docker', 'exec', "${->startDockerContainer.containerId()}", '/bin/bash', '-c', """echo '[params]\n branch_repo = "${getBranchRepo()}"' > /tmp/_config_branch_repo.toml""" } From 393f1bacffa64360d359b646655ead3e090373f4 Mon Sep 17 00:00:00 2001 From: Ankur Goenka Date: Thu, 21 May 2020 15:03:38 -0700 Subject: [PATCH 050/151] [BEAM-10060] Upgrading Python containers to beam-master-20200521 --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index 9be60ff497fa..446f00cda49b 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -40,10 +40,10 @@ # Update this version to the next version whenever there is a change that will # require changes to legacy Dataflow worker execution environment. -BEAM_CONTAINER_VERSION = 'beam-master-20200518' +BEAM_CONTAINER_VERSION = 'beam-master-20200521' # Update this version to the next version whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20200518' +BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20200521' # TODO(BEAM-5939): Remove these shared names once Dataflow worker is updated. PICKLED_MAIN_SESSION_FILE = 'pickled_main_session' From 73fa1356e4bc2d04e51c06cbf7c3aa860264f9ee Mon Sep 17 00:00:00 2001 From: Dariusz Aniszewski Date: Fri, 22 May 2020 00:39:16 +0200 Subject: [PATCH 051/151] [BEAM-9722] added SnowflakeIO with Read operation (#11360) * [BEAM-9722] added SnowflakeIO with Read operation * [BEAM-9722] Added SnowflakeCloudProvider to enable use various clouds with Snowflake * [BEAM-9722] added docstrings for public methods * [BEAM-9722] Added changed cleanup staged GCS files to Beam FileSystems * [BEAM-9722] Added javadocs for public methods in DataSourceConfiguration * add testing p8 file to RAT exclude refactor SnowflakeCredentials add information about possibly left files on cloud storage small docs changes * documentation changes * [BEAM-9722] Added TestRule and changed Unit tests to use pipeline.run * [BEAM-9722] Renamed Snowflake Read unit test and applied spotless * [BEAM-9722] remove SnowflakeCloudProvider interface * [BEAM-9722] doc changes * [BEAM-9722] add `withoutValidation` to disable verifying connection to Snowflake during pipeline construction * [BEAM-9722] added MoveOption and removed leftover file * [BEAM-9722] fixed tests. Add tests for `withQuery` * [BEAM-9722] make `CopyIntoStageFn` retryable * [BEAM-9722] added `Reshuffle` step after `CopyIntoStageFn` Co-authored-by: Kasia Kucharczyk Co-authored-by: pawel.urbanowicz --- CHANGES.md | 1 + build.gradle | 3 + sdks/java/io/snowflake/build.gradle | 42 + .../beam/sdk/io/snowflake/CloudProvider.java | 32 + .../beam/sdk/io/snowflake/SnowflakeIO.java | 759 ++++++++++++++++++ .../snowflake/SnowflakePipelineOptions.java | 133 +++ .../sdk/io/snowflake/SnowflakeService.java | 36 + .../io/snowflake/SnowflakeServiceImpl.java | 90 +++ .../KeyPairSnowflakeCredentials.java | 81 ++ .../OAuthTokenSnowflakeCredentials.java | 31 + .../credentials/SnowflakeCredentials.java | 24 + .../SnowflakeCredentialsFactory.java | 55 ++ .../UsernamePasswordSnowflakeCredentials.java | 37 + .../snowflake/credentials/package-info.java | 20 + .../beam/sdk/io/snowflake/package-info.java | 20 + .../test/FakeSnowflakeBasicDataSource.java | 298 +++++++ .../snowflake/test/FakeSnowflakeDatabase.java | 81 ++ .../test/FakeSnowflakeServiceImpl.java | 64 ++ .../beam/sdk/io/snowflake/test/TestUtils.java | 40 + .../sdk/io/snowflake/test/package-info.java | 20 + .../test/unit/BatchTestPipelineOptions.java | 28 + .../unit/DataSourceConfigurationTest.java | 159 ++++ .../KeyPairSnowflakeCredentialsTest.java | 38 + .../OAuthTokenSnowflakeCredentialsTest.java | 46 ++ .../SnowflakeCredentialsFactoryTest.java | 77 ++ ...rnamePasswordSnowflakeCredentialsTest.java | 50 ++ .../test/unit/read/SnowflakeIOReadTest.java | 278 +++++++ .../src/test/resources/test_rsa_key.p8 | 29 + settings.gradle | 1 + 29 files changed, 2573 insertions(+) create mode 100644 sdks/java/io/snowflake/build.gradle create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/CloudProvider.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeService.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeServiceImpl.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/KeyPairSnowflakeCredentials.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/OAuthTokenSnowflakeCredentials.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentials.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/UsernamePasswordSnowflakeCredentials.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/package-info.java create mode 100644 sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/package-info.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/package-info.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/BatchTestPipelineOptions.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/KeyPairSnowflakeCredentialsTest.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/OAuthTokenSnowflakeCredentialsTest.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/SnowflakeCredentialsFactoryTest.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/UsernamePasswordSnowflakeCredentialsTest.java create mode 100644 sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java create mode 100644 sdks/java/io/snowflake/src/test/resources/test_rsa_key.p8 diff --git a/CHANGES.md b/CHANGES.md index 107e7dc8dff1..59082149617a 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -57,6 +57,7 @@ ## I/Os * Support for X source added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* Support for reading from Snowflake added (Java) ([BEAM-9722](https://issues.apache.org/jira/browse/BEAM-9722)). ## New Features / Improvements diff --git a/build.gradle b/build.gradle index c26a02067da3..65363b7569dc 100644 --- a/build.gradle +++ b/build.gradle @@ -112,6 +112,9 @@ rat { "learning/katas/**/task-remote-info.yaml", "learning/katas/*/IO/**/*.txt", + // test p8 file for SnowflakeIO + "sdks/java/io/snowflake/src/test/resources/test_rsa_key.p8", + // Mockito extensions "sdks/java/io/amazon-web-services2/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker", "sdks/java/extensions/ml/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker" diff --git a/sdks/java/io/snowflake/build.gradle b/sdks/java/io/snowflake/build.gradle new file mode 100644 index 000000000000..c51c0348efc0 --- /dev/null +++ b/sdks/java/io/snowflake/build.gradle @@ -0,0 +1,42 @@ +/* + * 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. + */ +plugins { id 'org.apache.beam.module' } +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.snowflake') +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() +description = "Apache Beam :: SDKs :: Java :: IO :: Snowflake" +ext.summary = "IO to read and write on Snowflake." +dependencies { + compile library.java.vendored_guava_26_0_jre + compile project(path: ":sdks:java:core", configuration: "shadow") + compile project(path: ":sdks:java:extensions:google-cloud-platform-core") + compile library.java.slf4j_api + compile group: 'net.snowflake', name: 'snowflake-jdbc', version: '3.11.0' + compile group: 'com.opencsv', name: 'opencsv', version: '5.0' + testCompile project(path: ":sdks:java:core", configuration: "shadowTest") + testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime") + testCompile project(path: ":sdks:java:testing:test-utils", configuration: "testRuntime") + testCompile library.java.avro + testCompile library.java.junit + testCompile library.java.hamcrest_core + testCompile library.java.hamcrest_library + testCompile library.java.slf4j_api + testRuntimeOnly library.java.hadoop_client + testRuntimeOnly library.java.slf4j_jdk14 + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/CloudProvider.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/CloudProvider.java new file mode 100644 index 000000000000..404859c96d81 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/CloudProvider.java @@ -0,0 +1,32 @@ +/* + * 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.beam.sdk.io.snowflake; + +public enum CloudProvider { + GCS("gs://"); + + private final String prefix; + + private CloudProvider(String prefix) { + this.prefix = prefix; + } + + public String getPrefix() { + return prefix; + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java new file mode 100644 index 000000000000..a67ba320d361 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java @@ -0,0 +1,759 @@ +/* + * 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.beam.sdk.io.snowflake; + +import static org.apache.beam.sdk.io.TextIO.readFiles; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import com.opencsv.CSVParser; +import com.opencsv.CSVParserBuilder; +import java.io.IOException; +import java.io.Serializable; +import java.security.PrivateKey; +import java.sql.Connection; +import java.sql.SQLException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import javax.sql.DataSource; +import net.snowflake.client.jdbc.SnowflakeBasicDataSource; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MoveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.io.snowflake.credentials.KeyPairSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.UsernamePasswordSnowflakeCredentials; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Wait; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * IO to read and write data on Snowflake. + * + *

    SnowflakeIO uses Snowflake + * JDBC driver under the hood, but data isn't read/written using JDBC directly. Instead, + * SnowflakeIO uses dedicated COPY operations to read/write data from/to a cloud bucket. By + * now only Google Cloud Storage is supported. + * + *

    To configure SnowflakeIO to read/write from your Snowflake instance, you have to provide a + * {@link DataSourceConfiguration} using {@link + * DataSourceConfiguration#create(SnowflakeCredentials)}, where {@link SnowflakeCredentials might be + * created using {@link org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory}}. + * Additionally one of {@link DataSourceConfiguration#withServerName(String)} or {@link + * DataSourceConfiguration#withUrl(String)} must be used to tell SnowflakeIO which instance to use. + *
    + * There are also other options available to configure connection to Snowflake: + * + *

      + *
    • {@link DataSourceConfiguration#withWarehouse(String)} to specify which Warehouse to use + *
    • {@link DataSourceConfiguration#withDatabase(String)} to specify which Database to connect + * to + *
    • {@link DataSourceConfiguration#withSchema(String)} to specify which schema to use + *
    • {@link DataSourceConfiguration#withRole(String)} to specify which role to use + *
    • {@link DataSourceConfiguration#withLoginTimeout(Integer)} to specify the timeout for the + * login + *
    • {@link DataSourceConfiguration#withPortNumber(Integer)} to specify custom port of Snowflake + * instance + *
    + * + *

    For example: + * + *

    {@code
    + * SnowflakeIO.DataSourceConfiguration dataSourceConfiguration =
    + *     SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
    + *         .withServerName(options.getServerName())
    + *         .withWarehouse(options.getWarehouse())
    + *         .withDatabase(options.getDatabase())
    + *         .withSchema(options.getSchema());
    + * }
    + * + *

    Reading from Snowflake

    + * + *

    SnowflakeIO.Read returns a bounded collection of {@code T} as a {@code PCollection}. T is + * the type returned by the provided {@link CsvMapper}. + * + *

    For example + * + *

    {@code
    + * PCollection items = pipeline.apply(
    + *  SnowflakeIO.read()
    + *    .withDataSourceConfiguration(dataSourceConfiguration)
    + *    .fromQuery(QUERY)
    + *    .withStagingBucketName(stagingBucketName)
    + *    .withIntegrationName(integrationName)
    + *    .withCsvMapper(...)
    + *    .withCoder(...));
    + * }
    + * + *

    Important When reading data from Snowflake, temporary CSV files are created on the + * specified stagingBucketName in directory named `sf_copy_csv_[RANDOM CHARS]_[TIMESTAMP]`. This + * directory and all the files are cleaned up automatically by default, but in case of failed + * pipeline they may remain and will have to be cleaned up manually. + */ +public class SnowflakeIO { + private static final Logger LOG = LoggerFactory.getLogger(SnowflakeIO.class); + + private static final String CSV_QUOTE_CHAR = "'"; + /** + * Read data from Snowflake. + * + * @param snowflakeService user-defined {@link SnowflakeService} + * @param Type of the data to be read. + */ + public static Read read(SnowflakeService snowflakeService) { + return new AutoValue_SnowflakeIO_Read.Builder() + .setSnowflakeService(snowflakeService) + .build(); + } + + /** + * Read data from Snowflake. + * + * @param Type of the data to be read. + */ + public static Read read() { + return read(new SnowflakeServiceImpl()); + } + + /** + * Interface for user-defined function mapping parts of CSV line into T. Used for + * SnowflakeIO.Read. + * + * @param Type of data to be read. + */ + @FunctionalInterface + public interface CsvMapper extends Serializable { + T mapRow(String[] parts) throws Exception; + } + + /** Implementation of {@link #read()}. */ + @AutoValue + public abstract static class Read extends PTransform> { + @Nullable + abstract SerializableFunction getDataSourceProviderFn(); + + @Nullable + abstract String getQuery(); + + @Nullable + abstract String getTable(); + + @Nullable + abstract String getIntegrationName(); + + @Nullable + abstract String getStagingBucketName(); + + @Nullable + abstract CsvMapper getCsvMapper(); + + @Nullable + abstract Coder getCoder(); + + @Nullable + abstract SnowflakeService getSnowflakeService(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setDataSourceProviderFn( + SerializableFunction dataSourceProviderFn); + + abstract Builder setQuery(String query); + + abstract Builder setTable(String table); + + abstract Builder setIntegrationName(String integrationName); + + abstract Builder setStagingBucketName(String stagingBucketName); + + abstract Builder setCsvMapper(CsvMapper csvMapper); + + abstract Builder setCoder(Coder coder); + + abstract Builder setSnowflakeService(SnowflakeService snowflakeService); + + abstract Read build(); + } + + /** + * Setting information about Snowflake server. + * + * @param config - An instance of {@link DataSourceConfiguration}. + */ + public Read withDataSourceConfiguration(final DataSourceConfiguration config) { + if (config.getValidate()) { + try { + Connection connection = config.buildDatasource().getConnection(); + connection.close(); + } catch (SQLException e) { + throw new IllegalArgumentException( + "Invalid DataSourceConfiguration. Underlying cause: " + e); + } + } + return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config)); + } + + /** + * Setting function that will provide {@link DataSourceConfiguration} in runtime. + * + * @param dataSourceProviderFn a {@link SerializableFunction}. + */ + public Read withDataSourceProviderFn( + SerializableFunction dataSourceProviderFn) { + return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build(); + } + + /** + * A query to be executed in Snowflake. + * + * @param query - String with query. + */ + public Read fromQuery(String query) { + return toBuilder().setQuery(query).build(); + } + + /** + * A table name to be read in Snowflake. + * + * @param table - String with the name of the table. + */ + public Read fromTable(String table) { + return toBuilder().setTable(table).build(); + } + + /** + * Name of the cloud bucket (GCS by now) to use as tmp location of CSVs during COPY statement. + * + * @param stagingBucketName - String with the name of the bucket. + */ + public Read withStagingBucketName(String stagingBucketName) { + return toBuilder().setStagingBucketName(stagingBucketName).build(); + } + + /** + * Name of the Storage Integration in Snowflake to be used. See + * https://docs.snowflake.com/en/sql-reference/sql/create-storage-integration.html for + * reference. + * + * @param integrationName - String with the name of the Storage Integration. + */ + public Read withIntegrationName(String integrationName) { + return toBuilder().setIntegrationName(integrationName).build(); + } + + /** + * User-defined function mapping CSV lines into user data. + * + * @param csvMapper - an instance of {@link CsvMapper}. + */ + public Read withCsvMapper(CsvMapper csvMapper) { + return toBuilder().setCsvMapper(csvMapper).build(); + } + + /** + * A Coder to be used by the output PCollection generated by the source. + * + * @param coder - an instance of {@link Coder}. + */ + public Read withCoder(Coder coder) { + return toBuilder().setCoder(coder).build(); + } + + @Override + public PCollection expand(PBegin input) { + // Either table or query is required. If query is present, it's being used, table is used + // otherwise + checkArgument( + getQuery() != null || getTable() != null, "fromTable() or fromQuery() is required"); + checkArgument( + !(getQuery() != null && getTable() != null), + "fromTable() and fromQuery() are not allowed together"); + checkArgument(getCsvMapper() != null, "withCsvMapper() is required"); + checkArgument(getCoder() != null, "withCoder() is required"); + checkArgument(getIntegrationName() != null, "withIntegrationName() is required"); + checkArgument(getStagingBucketName() != null, "withStagingBucketName() is required"); + checkArgument( + (getDataSourceProviderFn() != null), + "withDataSourceConfiguration() or withDataSourceProviderFn() is required"); + + String tmpDirName = makeTmpDirName(); + String stagingBucketDir = String.format("%s/%s", getStagingBucketName(), tmpDirName); + + PCollection emptyCollection = input.apply(Create.of((Void) null)); + + PCollection output = + emptyCollection + .apply( + ParDo.of( + new CopyIntoStageFn( + getDataSourceProviderFn(), + getQuery(), + getTable(), + getIntegrationName(), + stagingBucketDir, + getSnowflakeService()))) + .apply(Reshuffle.viaRandomKey()) + .apply(FileIO.matchAll()) + .apply(FileIO.readMatches()) + .apply(readFiles()) + .apply(ParDo.of(new MapCsvToStringArrayFn())) + .apply(ParDo.of(new MapStringArrayToUserDataFn<>(getCsvMapper()))); + + output.setCoder(getCoder()); + + emptyCollection + .apply(Wait.on(output)) + .apply(ParDo.of(new CleanTmpFilesFromGcsFn(stagingBucketDir))); + + return output; + } + + private String makeTmpDirName() { + return String.format( + "sf_copy_csv_%s_%s", + new SimpleDateFormat("yyyyMMdd_HHmmss").format(new Date()), + UUID.randomUUID().toString().subSequence(0, 8) // first 8 chars of UUID should be enough + ); + } + + private static class CopyIntoStageFn extends DoFn { + private final SerializableFunction dataSourceProviderFn; + private final String query; + private final String table; + private final String integrationName; + private final String stagingBucketDir; + private final SnowflakeService snowflakeService; + + private CopyIntoStageFn( + SerializableFunction dataSourceProviderFn, + String query, + String table, + String integrationName, + String stagingBucketDir, + SnowflakeService snowflakeService) { + this.dataSourceProviderFn = dataSourceProviderFn; + this.query = query; + this.table = table; + this.integrationName = integrationName; + this.stagingBucketDir = + String.format( + "%s/run_%s/", stagingBucketDir, UUID.randomUUID().toString().subSequence(0, 8)); + this.snowflakeService = snowflakeService; + } + + @ProcessElement + public void processElement(ProcessContext context) throws Exception { + String output = + snowflakeService.copyIntoStage( + dataSourceProviderFn, query, table, integrationName, stagingBucketDir); + + context.output(output); + } + } + + public static class MapCsvToStringArrayFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + String csvLine = c.element(); + CSVParser parser = new CSVParserBuilder().withQuoteChar(CSV_QUOTE_CHAR.charAt(0)).build(); + String[] parts = parser.parseLine(csvLine); + c.output(parts); + } + } + + private static class MapStringArrayToUserDataFn extends DoFn { + private final CsvMapper csvMapper; + + public MapStringArrayToUserDataFn(CsvMapper csvMapper) { + this.csvMapper = csvMapper; + } + + @ProcessElement + public void processElement(ProcessContext context) throws Exception { + context.output(csvMapper.mapRow(context.element())); + } + } + + public static class CleanTmpFilesFromGcsFn extends DoFn { + private final String stagingBucketDir; + + public CleanTmpFilesFromGcsFn(String stagingBucketDir) { + this.stagingBucketDir = stagingBucketDir; + } + + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + String combinedPath = stagingBucketDir + "/**"; + List paths = + FileSystems.match(combinedPath).metadata().stream() + .map(metadata -> metadata.resourceId()) + .collect(Collectors.toList()); + + FileSystems.delete(paths, MoveOptions.StandardMoveOptions.IGNORE_MISSING_FILES); + } + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + if (getQuery() != null) { + builder.add(DisplayData.item("query", getQuery())); + } + if (getTable() != null) { + builder.add(DisplayData.item("table", getTable())); + } + builder.add(DisplayData.item("integrationName", getIntegrationName())); + builder.add(DisplayData.item("stagingBucketName", getStagingBucketName())); + builder.add(DisplayData.item("csvMapper", getCsvMapper().getClass().getName())); + builder.add(DisplayData.item("coder", getCoder().getClass().getName())); + if (getDataSourceProviderFn() instanceof HasDisplayData) { + ((HasDisplayData) getDataSourceProviderFn()).populateDisplayData(builder); + } + } + } + + /** + * A POJO describing a {@link DataSource}, providing all properties allowing to create a {@link + * DataSource}. + */ + @AutoValue + public abstract static class DataSourceConfiguration implements Serializable { + @Nullable + public abstract String getUrl(); + + @Nullable + public abstract String getUsername(); + + @Nullable + public abstract String getPassword(); + + @Nullable + public abstract PrivateKey getPrivateKey(); + + @Nullable + public abstract String getOauthToken(); + + @Nullable + public abstract String getDatabase(); + + @Nullable + public abstract String getWarehouse(); + + @Nullable + public abstract String getSchema(); + + @Nullable + public abstract String getServerName(); + + @Nullable + public abstract Integer getPortNumber(); + + @Nullable + public abstract String getRole(); + + @Nullable + public abstract Integer getLoginTimeout(); + + @Nullable + public abstract Boolean getSsl(); + + @Nullable + public abstract Boolean getValidate(); + + @Nullable + public abstract DataSource getDataSource(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setUrl(String url); + + abstract Builder setUsername(String username); + + abstract Builder setPassword(String password); + + abstract Builder setPrivateKey(PrivateKey privateKey); + + abstract Builder setOauthToken(String oauthToken); + + abstract Builder setDatabase(String database); + + abstract Builder setWarehouse(String warehouse); + + abstract Builder setSchema(String schema); + + abstract Builder setServerName(String serverName); + + abstract Builder setPortNumber(Integer portNumber); + + abstract Builder setRole(String role); + + abstract Builder setLoginTimeout(Integer loginTimeout); + + abstract Builder setSsl(Boolean ssl); + + abstract Builder setValidate(Boolean validate); + + abstract Builder setDataSource(DataSource dataSource); + + abstract DataSourceConfiguration build(); + } + + /** + * Creates {@link DataSourceConfiguration} from existing instance of {@link DataSource}. + * + * @param dataSource - an instance of {@link DataSource}. + */ + public static DataSourceConfiguration create(DataSource dataSource) { + checkArgument(dataSource instanceof Serializable, "dataSource must be Serializable"); + return new AutoValue_SnowflakeIO_DataSourceConfiguration.Builder() + .setValidate(true) + .setDataSource(dataSource) + .build(); + } + + /** + * Creates {@link DataSourceConfiguration} from instance of {@link SnowflakeCredentials}. + * + * @param credentials - an instance of {@link SnowflakeCredentials}. + */ + public static DataSourceConfiguration create(SnowflakeCredentials credentials) { + if (credentials instanceof UsernamePasswordSnowflakeCredentials) { + return new AutoValue_SnowflakeIO_DataSourceConfiguration.Builder() + .setValidate(true) + .setUsername(((UsernamePasswordSnowflakeCredentials) credentials).getUsername()) + .setPassword(((UsernamePasswordSnowflakeCredentials) credentials).getPassword()) + .build(); + } else if (credentials instanceof OAuthTokenSnowflakeCredentials) { + return new AutoValue_SnowflakeIO_DataSourceConfiguration.Builder() + .setValidate(true) + .setOauthToken(((OAuthTokenSnowflakeCredentials) credentials).getToken()) + .build(); + } else if (credentials instanceof KeyPairSnowflakeCredentials) { + return new AutoValue_SnowflakeIO_DataSourceConfiguration.Builder() + .setValidate(true) + .setUsername(((KeyPairSnowflakeCredentials) credentials).getUsername()) + .setPrivateKey(((KeyPairSnowflakeCredentials) credentials).getPrivateKey()) + .build(); + } + throw new IllegalArgumentException( + "Can't create DataSourceConfiguration from given credentials"); + } + + /** + * Sets URL of Snowflake server in following format: + * jdbc:snowflake://.snowflakecomputing.com + * + *

    Either withUrl or withServerName is required. + * + * @param url - String with URL of the Snowflake server. + */ + public DataSourceConfiguration withUrl(String url) { + checkArgument( + url.startsWith("jdbc:snowflake://"), + "url must have format: jdbc:snowflake://.snowflakecomputing.com"); + checkArgument( + url.endsWith("snowflakecomputing.com"), + "url must have format: jdbc:snowflake://.snowflakecomputing.com"); + return builder().setUrl(url).build(); + } + + /** + * Sets database to use. + * + * @param database - String with database name. + */ + public DataSourceConfiguration withDatabase(String database) { + return builder().setDatabase(database).build(); + } + + /** + * Sets Snowflake Warehouse to use. + * + * @param warehouse - String with warehouse name. + */ + public DataSourceConfiguration withWarehouse(String warehouse) { + return builder().setWarehouse(warehouse).build(); + } + + /** + * Sets schema to use when connecting to Snowflake. + * + * @param schema - String with schema name. + */ + public DataSourceConfiguration withSchema(String schema) { + return builder().setSchema(schema).build(); + } + + /** + * Sets the name of the Snowflake server. Following format is required: + * .snowflakecomputing.com + * + *

    Either withServerName or withUrl is required. + * + * @param serverName - String with server name. + */ + public DataSourceConfiguration withServerName(String serverName) { + checkArgument( + serverName.endsWith("snowflakecomputing.com"), + "serverName must be in format .snowflakecomputing.com"); + return builder().setServerName(serverName).build(); + } + + /** + * Sets port number to use to connect to Snowflake. + * + * @param portNumber - Integer with port number. + */ + public DataSourceConfiguration withPortNumber(Integer portNumber) { + return builder().setPortNumber(portNumber).build(); + } + + /** + * Sets user's role to be used when running queries on Snowflake. + * + * @param role - String with role name. + */ + public DataSourceConfiguration withRole(String role) { + return builder().setRole(role).build(); + } + + /** + * Sets loginTimeout that will be used in {@link SnowflakeBasicDataSource:setLoginTimeout}. + * + * @param loginTimeout - Integer with timeout value. + */ + public DataSourceConfiguration withLoginTimeout(Integer loginTimeout) { + return builder().setLoginTimeout(loginTimeout).build(); + } + + /** + * Disables validation of connection parameters prior to pipeline submission. + * + * @return + */ + public DataSourceConfiguration withoutValidation() { + return builder().setValidate(false).build(); + } + + void populateDisplayData(DisplayData.Builder builder) { + if (getDataSource() != null) { + builder.addIfNotNull(DisplayData.item("dataSource", getDataSource().getClass().getName())); + } else { + builder.addIfNotNull(DisplayData.item("jdbcUrl", getUrl())); + builder.addIfNotNull(DisplayData.item("username", getUsername())); + } + } + + /** Builds {@link SnowflakeBasicDataSource} based on the current configuration. */ + public DataSource buildDatasource() { + if (getDataSource() == null) { + SnowflakeBasicDataSource basicDataSource = new SnowflakeBasicDataSource(); + + if (getUrl() != null) { + basicDataSource.setUrl(getUrl()); + } + if (getUsername() != null) { + basicDataSource.setUser(getUsername()); + } + if (getPassword() != null) { + basicDataSource.setPassword(getPassword()); + } + if (getPrivateKey() != null) { + basicDataSource.setPrivateKey(getPrivateKey()); + } + if (getDatabase() != null) { + basicDataSource.setDatabaseName(getDatabase()); + } + if (getWarehouse() != null) { + basicDataSource.setWarehouse(getWarehouse()); + } + if (getSchema() != null) { + basicDataSource.setSchema(getSchema()); + } + if (getServerName() != null) { + basicDataSource.setServerName(getServerName()); + } + if (getPortNumber() != null) { + basicDataSource.setPortNumber(getPortNumber()); + } + if (getRole() != null) { + basicDataSource.setRole(getRole()); + } + if (getLoginTimeout() != null) { + try { + basicDataSource.setLoginTimeout(getLoginTimeout()); + } catch (SQLException e) { + throw new RuntimeException("Failed to setLoginTimeout"); + } + } + if (getOauthToken() != null) { + basicDataSource.setOauthToken(getOauthToken()); + } + return basicDataSource; + } + return getDataSource(); + } + } + + public static class DataSourceProviderFromDataSourceConfiguration + implements SerializableFunction, HasDisplayData { + private static final ConcurrentHashMap instances = + new ConcurrentHashMap<>(); + private final DataSourceConfiguration config; + + private DataSourceProviderFromDataSourceConfiguration(DataSourceConfiguration config) { + this.config = config; + } + + public static SerializableFunction of(DataSourceConfiguration config) { + return new DataSourceProviderFromDataSourceConfiguration(config); + } + + @Override + public DataSource apply(Void input) { + return instances.computeIfAbsent(config, (config) -> config.buildDatasource()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + config.populateDisplayData(builder); + } + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.java new file mode 100644 index 000000000000..783230edbb8d --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakePipelineOptions.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.beam.sdk.io.snowflake; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +public interface SnowflakePipelineOptions extends PipelineOptions { + String BASIC_CONNECTION_INFO_VALIDATION_GROUP = "BASIC_CONNECTION_INFO_GROUP"; + String AUTH_VALIDATION_GROUP = "AUTH_VALIDATION_GROUP"; + + @Description( + "Snowflake's JDBC-like url including account name and region without any parameters.") + @Validation.Required(groups = BASIC_CONNECTION_INFO_VALIDATION_GROUP) + String getUrl(); + + void setUrl(String url); + + @Description("Server Name - full server name with account, zone and domain.") + @Validation.Required(groups = BASIC_CONNECTION_INFO_VALIDATION_GROUP) + String getServerName(); + + void setServerName(String serverName); + + @Description("Username. Required for username/password and Private Key authentication.") + @Validation.Required(groups = AUTH_VALIDATION_GROUP) + String getUsername(); + + void setUsername(String username); + + @Description("OAuth token. Required for OAuth authentication only.") + @Validation.Required(groups = AUTH_VALIDATION_GROUP) + String getOauthToken(); + + void setOauthToken(String oauthToken); + + @Description("Password. Required for username/password authentication only.") + @Default.String("") + String getPassword(); + + void setPassword(String password); + + @Description("Path to Private Key file. Required for Private Key authentication only.") + @Default.String("") + String getPrivateKeyPath(); + + void setPrivateKeyPath(String privateKeyPath); + + @Description("Private Key's passphrase. Required for Private Key authentication only.") + @Default.String("") + String getPrivateKeyPassphrase(); + + void setPrivateKeyPassphrase(String keyPassphrase); + + @Description("Warehouse to use. Optional.") + @Default.String("") + String getWarehouse(); + + void setWarehouse(String warehouse); + + @Description("Database name to connect to. Optional.") + @Default.String("") + String getDatabase(); + + void setDatabase(String database); + + @Description("Schema to use. Optional.") + @Default.String("") + String getSchema(); + + void setSchema(String schema); + + @Description("Role to use. Optional.") + @Default.String("") + String getRole(); + + void setRole(String role); + + @Description("Authenticator to use. Optional.") + @Default.String("") + String getAuthenticator(); + + void setAuthenticator(String authenticator); + + @Description("Port number. Optional.") + @Default.String("") + String getPortNumber(); + + void setPortNumber(String portNumber); + + @Description("Login timeout. Optional.") + @Default.String("") + String getLoginTimeout(); + + void setLoginTimeout(String loginTimeout); + + @Description("External location name to connect to.") + String getExternalLocation(); + + void setExternalLocation(String externalLocation); + + @Description("Temporary GCS bucket name") + String getStagingBucketName(); + + void setStagingBucketName(String stagingBucketName); + + @Description("Storage integration - required in case the external stage is not specified.") + String getStorageIntegration(); + + void setStorageIntegration(String integration); + + @Description("Stage name. Optional.") + String getStage(); + + void setStage(String stage); +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeService.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeService.java new file mode 100644 index 000000000000..6375e79e74ce --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeService.java @@ -0,0 +1,36 @@ +/* + * 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.beam.sdk.io.snowflake; + +import java.io.Serializable; +import java.sql.SQLException; +import javax.sql.DataSource; +import org.apache.beam.sdk.transforms.SerializableFunction; + +/** Interface which defines common methods for interacting with SnowFlake. */ +public interface SnowflakeService extends Serializable { + String CSV_QUOTE_CHAR_FOR_COPY = "''"; + + String copyIntoStage( + SerializableFunction dataSourceProviderFn, + String query, + String table, + String integrationName, + String stagingBucketDir) + throws SQLException; +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeServiceImpl.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeServiceImpl.java new file mode 100644 index 000000000000..5aaad06b7ed2 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeServiceImpl.java @@ -0,0 +1,90 @@ +/* + * 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.beam.sdk.io.snowflake; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.function.Consumer; +import javax.sql.DataSource; +import org.apache.beam.sdk.transforms.SerializableFunction; + +/** + * Implemenation of {@link org.apache.beam.sdk.io.snowflake.SnowflakeService} used in production. + */ +public class SnowflakeServiceImpl implements SnowflakeService { + private static final String SNOWFLAKE_GCS_PREFIX = "gcs://"; + + @Override + public String copyIntoStage( + SerializableFunction dataSourceProviderFn, + String query, + String table, + String integrationName, + String stagingBucketDir) + throws SQLException { + + String from; + if (query != null) { + // Query must be surrounded with brackets + from = String.format("(%s)", query); + } else { + from = table; + } + + String copyQuery = + String.format( + "COPY INTO '%s' FROM %s STORAGE_INTEGRATION=%s FILE_FORMAT=(TYPE=CSV COMPRESSION=GZIP FIELD_OPTIONALLY_ENCLOSED_BY='%s');", + getProperBucketDir(stagingBucketDir), from, integrationName, CSV_QUOTE_CHAR_FOR_COPY); + + runStatement(copyQuery, getConnection(dataSourceProviderFn), null); + + return stagingBucketDir.concat("*"); + } + + private static void runStatement(String query, Connection connection, Consumer resultSetMethod) + throws SQLException { + PreparedStatement statement = connection.prepareStatement(query); + try { + if (resultSetMethod != null) { + ResultSet resultSet = statement.executeQuery(); + resultSetMethod.accept(resultSet); + } else { + statement.execute(); + } + } finally { + statement.close(); + connection.close(); + } + } + + private Connection getConnection(SerializableFunction dataSourceProviderFn) + throws SQLException { + DataSource dataSource = dataSourceProviderFn.apply(null); + return dataSource.getConnection(); + } + + // Snowflake is expecting "gcs://" prefix for GCS and Beam "gs://" + private String getProperBucketDir(String bucketDir) { + if (bucketDir.contains(CloudProvider.GCS.getPrefix())) { + return bucketDir.replace(CloudProvider.GCS.getPrefix(), SNOWFLAKE_GCS_PREFIX); + } + return bucketDir; + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/KeyPairSnowflakeCredentials.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/KeyPairSnowflakeCredentials.java new file mode 100644 index 000000000000..286ec628fc8b --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/KeyPairSnowflakeCredentials.java @@ -0,0 +1,81 @@ +/* + * 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.beam.sdk.io.snowflake.credentials; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.security.InvalidKeyException; +import java.security.KeyFactory; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.security.spec.InvalidKeySpecException; +import java.security.spec.PKCS8EncodedKeySpec; +import java.util.Base64; +import javax.crypto.EncryptedPrivateKeyInfo; +import javax.crypto.SecretKeyFactory; +import javax.crypto.spec.PBEKeySpec; + +/** POJO for handling Key-Pair authentication against Snowflake. */ +public class KeyPairSnowflakeCredentials implements SnowflakeCredentials { + private String username; + private PrivateKey privateKey; + + public KeyPairSnowflakeCredentials( + String username, String privateKeyPath, String privateKeyPassword) { + this.username = username; + this.privateKey = getPrivateKey(privateKeyPath, privateKeyPassword); + } + + public KeyPairSnowflakeCredentials(String username, PrivateKey privateKey) { + this.username = username; + this.privateKey = privateKey; + } + + private PrivateKey getPrivateKey(String privateKeyPath, String privateKeyPassphrase) { + try { + byte[] keyBytes = Files.readAllBytes(Paths.get(privateKeyPath)); + + String encrypted = new String(keyBytes, Charset.defaultCharset()); + encrypted = encrypted.replace("-----BEGIN ENCRYPTED PRIVATE KEY-----", ""); + encrypted = encrypted.replace("-----END ENCRYPTED PRIVATE KEY-----", ""); + EncryptedPrivateKeyInfo pkInfo = + new EncryptedPrivateKeyInfo(Base64.getMimeDecoder().decode(encrypted)); + PBEKeySpec keySpec = new PBEKeySpec(privateKeyPassphrase.toCharArray()); + SecretKeyFactory pbeKeyFactory = SecretKeyFactory.getInstance(pkInfo.getAlgName()); + PKCS8EncodedKeySpec encodedKeySpec = pkInfo.getKeySpec(pbeKeyFactory.generateSecret(keySpec)); + + KeyFactory keyFactory = KeyFactory.getInstance("RSA"); + return keyFactory.generatePrivate(encodedKeySpec); + } catch (IOException + | NoSuchAlgorithmException + | InvalidKeySpecException + | InvalidKeyException ex) { + throw new RuntimeException("Can't create PrivateKey from options"); + } + } + + public String getUsername() { + return username; + } + + public PrivateKey getPrivateKey() { + return privateKey; + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/OAuthTokenSnowflakeCredentials.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/OAuthTokenSnowflakeCredentials.java new file mode 100644 index 000000000000..be102a8b0f41 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/OAuthTokenSnowflakeCredentials.java @@ -0,0 +1,31 @@ +/* + * 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.beam.sdk.io.snowflake.credentials; + +/** POJO for handling OAuth authentication against Snowflake, using pre-obtained OAuth token. */ +public class OAuthTokenSnowflakeCredentials implements SnowflakeCredentials { + private String token; + + public OAuthTokenSnowflakeCredentials(String token) { + this.token = token; + } + + public String getToken() { + return token; + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentials.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentials.java new file mode 100644 index 000000000000..e3abf91f7d12 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentials.java @@ -0,0 +1,24 @@ +/* + * 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.beam.sdk.io.snowflake.credentials; + +/** + * Interface for holding credentials. Allows creating {@link + * org.apache.beam.sdk.io.snowflake.SnowflakeIO.DataSourceConfiguration}. + */ +public interface SnowflakeCredentials {} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java new file mode 100644 index 000000000000..3876c2f10d71 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/SnowflakeCredentialsFactory.java @@ -0,0 +1,55 @@ +/* + * 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.beam.sdk.io.snowflake.credentials; + +import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions; + +/** + * Factory class for creating implementations of {@link SnowflakeCredentials} from {@link + * SnowflakePipelineOptions}. + */ +public class SnowflakeCredentialsFactory { + public static SnowflakeCredentials of(SnowflakePipelineOptions options) { + if (oauthOptionsAvailable(options)) { + return new OAuthTokenSnowflakeCredentials(options.getOauthToken()); + } else if (usernamePasswordOptionsAvailable(options)) { + return new UsernamePasswordSnowflakeCredentials(options.getUsername(), options.getPassword()); + } else if (keyPairOptionsAvailable(options)) { + return new KeyPairSnowflakeCredentials( + options.getUsername(), options.getPrivateKeyPath(), options.getPrivateKeyPassphrase()); + } + throw new RuntimeException("Can't get credentials from Options"); + } + + private static boolean oauthOptionsAvailable(SnowflakePipelineOptions options) { + return options.getOauthToken() != null && !options.getOauthToken().isEmpty(); + } + + private static boolean usernamePasswordOptionsAvailable(SnowflakePipelineOptions options) { + return options.getUsername() != null + && !options.getUsername().isEmpty() + && !options.getPassword().isEmpty(); + } + + private static boolean keyPairOptionsAvailable(SnowflakePipelineOptions options) { + return options.getUsername() != null + && !options.getUsername().isEmpty() + && !options.getPrivateKeyPath().isEmpty() + && !options.getPrivateKeyPassphrase().isEmpty(); + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/UsernamePasswordSnowflakeCredentials.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/UsernamePasswordSnowflakeCredentials.java new file mode 100644 index 000000000000..1d8bdce0cc2e --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/UsernamePasswordSnowflakeCredentials.java @@ -0,0 +1,37 @@ +/* + * 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.beam.sdk.io.snowflake.credentials; + +/** POJO for handling Username & Password authentication against Snowflake. */ +public class UsernamePasswordSnowflakeCredentials implements SnowflakeCredentials { + private String username; + private String password; + + public UsernamePasswordSnowflakeCredentials(String username, String password) { + this.username = username; + this.password = password; + } + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } +} diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/package-info.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/package-info.java new file mode 100644 index 000000000000..f76d2419b8ce --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/credentials/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Credentials for SnowflakeIO. */ +package org.apache.beam.sdk.io.snowflake.credentials; diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/package-info.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/package-info.java new file mode 100644 index 000000000000..9dbcf05a4074 --- /dev/null +++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Snowflake IO transforms. */ +package org.apache.beam.sdk.io.snowflake; diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java new file mode 100644 index 000000000000..5fc694fb9d0c --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java @@ -0,0 +1,298 @@ +/* + * 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.beam.sdk.io.snowflake.test; + +import java.io.Serializable; +import java.sql.Array; +import java.sql.Blob; +import java.sql.CallableStatement; +import java.sql.Clob; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.NClob; +import java.sql.PreparedStatement; +import java.sql.SQLClientInfoException; +import java.sql.SQLException; +import java.sql.SQLWarning; +import java.sql.SQLXML; +import java.sql.Savepoint; +import java.sql.Statement; +import java.sql.Struct; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Executor; +import net.snowflake.client.jdbc.SnowflakeBasicDataSource; + +/** + * Fake implementation of {@link net.snowflake.client.jdbc.SnowflakeBasicDataSource} used in tests. + */ +public class FakeSnowflakeBasicDataSource extends SnowflakeBasicDataSource implements Serializable { + @Override + public FakeConnection getConnection() throws SQLException { + return new FakeConnection(); + } + + private class FakeConnection implements Connection { + + @Override + public Statement createStatement() throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement(String sql) throws SQLException { + return null; + } + + @Override + public CallableStatement prepareCall(String sql) throws SQLException { + return null; + } + + @Override + public String nativeSQL(String sql) throws SQLException { + return null; + } + + @Override + public void setAutoCommit(boolean autoCommit) throws SQLException {} + + @Override + public boolean getAutoCommit() throws SQLException { + return false; + } + + @Override + public void commit() throws SQLException {} + + @Override + public void rollback() throws SQLException {} + + @Override + public void close() throws SQLException {} + + @Override + public boolean isClosed() throws SQLException { + return false; + } + + @Override + public DatabaseMetaData getMetaData() throws SQLException { + return null; + } + + @Override + public void setReadOnly(boolean readOnly) throws SQLException {} + + @Override + public boolean isReadOnly() throws SQLException { + return false; + } + + @Override + public void setCatalog(String catalog) throws SQLException {} + + @Override + public String getCatalog() throws SQLException { + return null; + } + + @Override + public void setTransactionIsolation(int level) throws SQLException {} + + @Override + public int getTransactionIsolation() throws SQLException { + return 0; + } + + @Override + public SQLWarning getWarnings() throws SQLException { + return null; + } + + @Override + public void clearWarnings() throws SQLException {} + + @Override + public Statement createStatement(int resultSetType, int resultSetConcurrency) + throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement( + String sql, int resultSetType, int resultSetConcurrency) throws SQLException { + return null; + } + + @Override + public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) + throws SQLException { + return null; + } + + @Override + public Map> getTypeMap() throws SQLException { + return null; + } + + @Override + public void setTypeMap(Map> map) throws SQLException {} + + @Override + public void setHoldability(int holdability) throws SQLException {} + + @Override + public int getHoldability() throws SQLException { + return 0; + } + + @Override + public Savepoint setSavepoint() throws SQLException { + return null; + } + + @Override + public Savepoint setSavepoint(String name) throws SQLException { + return null; + } + + @Override + public void rollback(Savepoint savepoint) throws SQLException {} + + @Override + public void releaseSavepoint(Savepoint savepoint) throws SQLException {} + + @Override + public Statement createStatement( + int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement( + String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) + throws SQLException { + return null; + } + + @Override + public CallableStatement prepareCall( + String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) + throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) + throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException { + return null; + } + + @Override + public PreparedStatement prepareStatement(String sql, String[] columnNames) + throws SQLException { + return null; + } + + @Override + public Clob createClob() throws SQLException { + return null; + } + + @Override + public Blob createBlob() throws SQLException { + return null; + } + + @Override + public NClob createNClob() throws SQLException { + return null; + } + + @Override + public SQLXML createSQLXML() throws SQLException { + return null; + } + + @Override + public boolean isValid(int timeout) throws SQLException { + return false; + } + + @Override + public void setClientInfo(String name, String value) throws SQLClientInfoException {} + + @Override + public void setClientInfo(Properties properties) throws SQLClientInfoException {} + + @Override + public String getClientInfo(String name) throws SQLException { + return null; + } + + @Override + public Properties getClientInfo() throws SQLException { + return null; + } + + @Override + public Array createArrayOf(String typeName, Object[] elements) throws SQLException { + return null; + } + + @Override + public Struct createStruct(String typeName, Object[] attributes) throws SQLException { + return null; + } + + @Override + public void setSchema(String schema) throws SQLException {} + + @Override + public String getSchema() throws SQLException { + return null; + } + + @Override + public void abort(Executor executor) throws SQLException {} + + @Override + public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {} + + @Override + public int getNetworkTimeout() throws SQLException { + return 0; + } + + @Override + public T unwrap(Class iface) throws SQLException { + return null; + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + return false; + } + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java new file mode 100644 index 000000000000..5bf8b21afa18 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java @@ -0,0 +1,81 @@ +/* + * 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.beam.sdk.io.snowflake.test; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import net.snowflake.client.jdbc.SnowflakeSQLException; + +/** Fake implementation of SnowFlake warehouse used in test code. */ +public class FakeSnowflakeDatabase implements Serializable { + private static Map> tables = new HashMap<>(); + + private FakeSnowflakeDatabase() { + tables = new HashMap<>(); + } + + public static void createTable(String table) { + FakeSnowflakeDatabase.tables.put(table, Collections.emptyList()); + } + + public static List getElements(String table) throws SnowflakeSQLException { + if (!isTableExist(table)) { + throw new SnowflakeSQLException( + null, "SQL compilation error: Table does not exist", table, 0); + } + + return FakeSnowflakeDatabase.tables.get(table); + } + + public static List runQuery(String query) throws SnowflakeSQLException { + if (query.startsWith("SELECT * FROM ")) { + String tableName = query.replace("SELECT * FROM ", ""); + return getElements(tableName); + } + throw new SnowflakeSQLException(null, "SQL compilation error: Invalid query", query, 0); + } + + public static List getElementsAsLong(String table) throws SnowflakeSQLException { + List elements = getElements(table); + return elements.stream().map(Long::parseLong).collect(Collectors.toList()); + } + + public static boolean isTableExist(String table) { + return FakeSnowflakeDatabase.tables.containsKey(table); + } + + public static boolean isTableEmpty(String table) { + return FakeSnowflakeDatabase.tables.get(table).isEmpty(); + } + + public static void createTableWithElements(String table, List rows) { + FakeSnowflakeDatabase.tables.put(table, rows); + } + + public static void clean() { + FakeSnowflakeDatabase.tables = new HashMap<>(); + } + + public static void truncateTable(String table) { + FakeSnowflakeDatabase.createTable(table); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java new file mode 100644 index 000000000000..4a62dcd3181d --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeServiceImpl.java @@ -0,0 +1,64 @@ +/* + * 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.beam.sdk.io.snowflake.test; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.sql.SQLException; +import java.util.List; +import javax.sql.DataSource; +import org.apache.beam.sdk.io.snowflake.SnowflakeService; +import org.apache.beam.sdk.transforms.SerializableFunction; + +/** + * Fake implementation of {@link org.apache.beam.sdk.io.snowflake.SnowflakeService} used in tests. + */ +public class FakeSnowflakeServiceImpl implements SnowflakeService { + + @Override + public String copyIntoStage( + SerializableFunction dataSourceProviderFn, + String query, + String table, + String integrationName, + String stagingBucketName) + throws SQLException { + + if (table != null) { + writeToFile(stagingBucketName, FakeSnowflakeDatabase.getElements(table)); + } + if (query != null) { + writeToFile(stagingBucketName, FakeSnowflakeDatabase.runQuery(query)); + } + + return String.format("./%s/*", stagingBucketName); + } + + private void writeToFile(String stagingBucketNameTmp, List rows) { + Path filePath = Paths.get(String.format("./%s/table.csv.gz", stagingBucketNameTmp)); + try { + Files.createDirectories(filePath.getParent()); + Files.createFile(filePath); + Files.write(filePath, rows); + } catch (IOException e) { + throw new RuntimeException("Failed to create files", e); + } + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java new file mode 100644 index 000000000000..aab8d7de9660 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java @@ -0,0 +1,40 @@ +/* + * 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.beam.sdk.io.snowflake.test; + +import java.io.File; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestUtils { + + private static final Logger LOG = LoggerFactory.getLogger(TestUtils.class); + + private static final String PRIVATE_KEY_FILE_NAME = "test_rsa_key.p8"; + private static final String PRIVATE_KEY_PASSPHRASE = "snowflake"; + + public static String getPrivateKeyPath(Class klass) { + ClassLoader classLoader = klass.getClassLoader(); + File file = new File(classLoader.getResource(PRIVATE_KEY_FILE_NAME).getFile()); + return file.getAbsolutePath(); + } + + public static String getPrivateKeyPassphrase() { + return PRIVATE_KEY_PASSPHRASE; + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/package-info.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/package-info.java new file mode 100644 index 000000000000..2e2cf2f7645c --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Snowflake IO tests. */ +package org.apache.beam.sdk.io.snowflake.test; diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/BatchTestPipelineOptions.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/BatchTestPipelineOptions.java new file mode 100644 index 000000000000..3504c455851b --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/BatchTestPipelineOptions.java @@ -0,0 +1,28 @@ +/* + * 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.beam.sdk.io.snowflake.test.unit; + +import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions; +import org.apache.beam.sdk.options.Description; + +public interface BatchTestPipelineOptions extends SnowflakePipelineOptions { + @Description("Table name to connect to.") + String getTable(); + + void setTable(String table); +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java new file mode 100644 index 000000000000..98f22132c77f --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.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.beam.sdk.io.snowflake.test.unit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import javax.sql.DataSource; +import net.snowflake.client.jdbc.SnowflakeBasicDataSource; +import org.apache.beam.sdk.io.snowflake.SnowflakeIO; +import org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials; +import org.junit.Before; +import org.junit.Test; + +/** Unit tests for {@link org.apache.beam.sdk.io.snowflake.SnowflakeIO.DataSourceConfiguration}. */ +public class DataSourceConfigurationTest { + + private SnowflakeIO.DataSourceConfiguration configuration; + + @Before + public void setUp() { + configuration = + SnowflakeIO.DataSourceConfiguration.create( + new OAuthTokenSnowflakeCredentials("some-token")); + } + + @Test + public void testSettingUrlWithBadPrefix() { + assertThrows( + IllegalArgumentException.class, + () -> configuration.withUrl("account.snowflakecomputing.com")); + } + + @Test + public void testSettingUrlWithBadSuffix() { + assertThrows( + IllegalArgumentException.class, () -> configuration.withUrl("jdbc:snowflake://account")); + } + + @Test + public void testSettingStringUrl() { + String url = "jdbc:snowflake://account.snowflakecomputing.com"; + configuration = configuration.withUrl(url); + assertEquals(url, configuration.getUrl()); + } + + @Test + public void testSettingServerNameWithBadSuffix() { + assertThrows( + IllegalArgumentException.class, () -> configuration.withServerName("not.properly.ended")); + } + + @Test + public void testSettingStringServerName() { + String serverName = "account.snowflakecomputing.com"; + configuration = configuration.withServerName(serverName); + assertEquals(serverName, configuration.getServerName()); + } + + @Test + public void testSettingStringDatabase() { + String database = "dbname"; + configuration = configuration.withDatabase(database); + assertEquals(database, configuration.getDatabase()); + } + + @Test + public void testSettingStringWarehouse() { + String warehouse = "warehouse"; + configuration = configuration.withWarehouse(warehouse); + assertEquals(warehouse, configuration.getWarehouse()); + } + + @Test + public void testSettingStringSchema() { + String schema = "schema"; + configuration = configuration.withSchema(schema); + assertEquals(schema, configuration.getSchema()); + } + + @Test + public void testSettingStringRole() { + String role = "role"; + configuration = configuration.withRole(role); + assertEquals(role, configuration.getRole()); + } + + @Test + public void testSettingStringPortNumber() { + Integer portNumber = 1234; + configuration = configuration.withPortNumber(portNumber); + assertEquals(portNumber, configuration.getPortNumber()); + } + + @Test + public void testSettingStringLoginTimeout() { + Integer loginTimeout = 999; + configuration = configuration.withLoginTimeout(loginTimeout); + assertEquals(loginTimeout, configuration.getLoginTimeout()); + } + + @Test + public void testSettingValidate() { + configuration = configuration.withoutValidation(); + assertEquals(false, configuration.getValidate()); + } + + @Test + public void testDataSourceCreatedFromUrl() { + String url = "jdbc:snowflake://account.snowflakecomputing.com"; + configuration = configuration.withUrl(url); + + DataSource dataSource = configuration.buildDatasource(); + + assertEquals(SnowflakeBasicDataSource.class, dataSource.getClass()); + assertEquals(url, ((SnowflakeBasicDataSource) dataSource).getUrl()); + } + + @Test + public void testDataSourceCreatedFromServerName() { + String serverName = "account.snowflakecomputing.com"; + configuration = configuration.withServerName(serverName); + + DataSource dataSource = configuration.buildDatasource(); + + String expectedUrl = "jdbc:snowflake://account.snowflakecomputing.com"; + assertEquals(SnowflakeBasicDataSource.class, dataSource.getClass()); + assertEquals(expectedUrl, ((SnowflakeBasicDataSource) dataSource).getUrl()); + } + + @Test + public void testDataSourceCreatedFromServerNameAndPort() { + String serverName = "account.snowflakecomputing.com"; + int portNumber = 1234; + + configuration = configuration.withServerName(serverName); + configuration = configuration.withPortNumber(portNumber); + + DataSource dataSource = configuration.buildDatasource(); + assertEquals(SnowflakeBasicDataSource.class, dataSource.getClass()); + String expectedUrl = "jdbc:snowflake://account.snowflakecomputing.com:1234"; + assertEquals(expectedUrl, ((SnowflakeBasicDataSource) dataSource).getUrl()); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/KeyPairSnowflakeCredentialsTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/KeyPairSnowflakeCredentialsTest.java new file mode 100644 index 000000000000..b231a2dde665 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/KeyPairSnowflakeCredentialsTest.java @@ -0,0 +1,38 @@ +/* + * 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.beam.sdk.io.snowflake.test.unit.credentials; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import org.apache.beam.sdk.io.snowflake.credentials.KeyPairSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.test.TestUtils; +import org.junit.Test; + +public class KeyPairSnowflakeCredentialsTest { + @Test + public void testFilePathConstructor() { + KeyPairSnowflakeCredentials credentials = + new KeyPairSnowflakeCredentials( + "username", + TestUtils.getPrivateKeyPath(getClass()), + TestUtils.getPrivateKeyPassphrase()); + assertEquals("username", credentials.getUsername()); + assertNotNull(credentials.getPrivateKey()); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/OAuthTokenSnowflakeCredentialsTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/OAuthTokenSnowflakeCredentialsTest.java new file mode 100644 index 000000000000..a1dee76f1601 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/OAuthTokenSnowflakeCredentialsTest.java @@ -0,0 +1,46 @@ +/* + * 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.beam.sdk.io.snowflake.test.unit.credentials; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.io.snowflake.SnowflakeIO; +import org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials; +import org.junit.Test; + +public class OAuthTokenSnowflakeCredentialsTest { + + @Test + public void testConstructor() { + OAuthTokenSnowflakeCredentials credentials = new OAuthTokenSnowflakeCredentials("token"); + + assertEquals("token", credentials.getToken()); + } + + @Test + public void testBuildingDataSource() { + OAuthTokenSnowflakeCredentials credentials = new OAuthTokenSnowflakeCredentials("token"); + + SnowflakeIO.DataSourceConfiguration configuration = + SnowflakeIO.DataSourceConfiguration.create(credentials); + + assertEquals(credentials.getToken(), configuration.getOauthToken()); + assertTrue(configuration.getValidate()); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/SnowflakeCredentialsFactoryTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/SnowflakeCredentialsFactoryTest.java new file mode 100644 index 000000000000..f9f612d8f002 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/SnowflakeCredentialsFactoryTest.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.snowflake.test.unit.credentials; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.sdk.io.snowflake.SnowflakePipelineOptions; +import org.apache.beam.sdk.io.snowflake.credentials.KeyPairSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory; +import org.apache.beam.sdk.io.snowflake.credentials.UsernamePasswordSnowflakeCredentials; +import org.apache.beam.sdk.io.snowflake.test.TestUtils; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Test; + +public class SnowflakeCredentialsFactoryTest { + + @Test + public void usernamePasswordTest() { + SnowflakePipelineOptions options = PipelineOptionsFactory.as(SnowflakePipelineOptions.class); + options.setUsername("username"); + options.setPassword("password"); + + SnowflakeCredentials credentials = SnowflakeCredentialsFactory.of(options); + + assertEquals(UsernamePasswordSnowflakeCredentials.class, credentials.getClass()); + } + + @Test + public void oauthTokenTest() { + SnowflakePipelineOptions options = PipelineOptionsFactory.as(SnowflakePipelineOptions.class); + options.setOauthToken("token"); + + SnowflakeCredentials credentials = SnowflakeCredentialsFactory.of(options); + + assertEquals(OAuthTokenSnowflakeCredentials.class, credentials.getClass()); + } + + @Test + public void keyPairTest() { + SnowflakePipelineOptions options = PipelineOptionsFactory.as(SnowflakePipelineOptions.class); + System.out.println(TestUtils.getPrivateKeyPath(getClass())); + options.setUsername("username"); + options.setPrivateKeyPath(TestUtils.getPrivateKeyPath(getClass())); + options.setPrivateKeyPassphrase(TestUtils.getPrivateKeyPassphrase()); + + SnowflakeCredentials credentials = SnowflakeCredentialsFactory.of(options); + + assertEquals(KeyPairSnowflakeCredentials.class, credentials.getClass()); + } + + @Test + public void emptyOptionsTest() { + SnowflakePipelineOptions options = PipelineOptionsFactory.as(SnowflakePipelineOptions.class); + + Exception ex = + assertThrows(RuntimeException.class, () -> SnowflakeCredentialsFactory.of(options)); + assertEquals("Can't get credentials from Options", ex.getMessage()); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/UsernamePasswordSnowflakeCredentialsTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/UsernamePasswordSnowflakeCredentialsTest.java new file mode 100644 index 000000000000..0c7503a079b8 --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/credentials/UsernamePasswordSnowflakeCredentialsTest.java @@ -0,0 +1,50 @@ +/* + * 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.beam.sdk.io.snowflake.test.unit.credentials; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.io.snowflake.SnowflakeIO; +import org.apache.beam.sdk.io.snowflake.credentials.UsernamePasswordSnowflakeCredentials; +import org.junit.Test; + +public class UsernamePasswordSnowflakeCredentialsTest { + + @Test + public void testConstructor() { + UsernamePasswordSnowflakeCredentials credentials = + new UsernamePasswordSnowflakeCredentials("username", "password"); + + assertEquals("username", credentials.getUsername()); + assertEquals("password", credentials.getPassword()); + } + + @Test + public void testBuildingDataSource() { + UsernamePasswordSnowflakeCredentials credentials = + new UsernamePasswordSnowflakeCredentials("username", "password"); + + SnowflakeIO.DataSourceConfiguration configuration = + SnowflakeIO.DataSourceConfiguration.create(credentials); + + assertEquals("username", configuration.getUsername()); + assertEquals("password", configuration.getPassword()); + assertTrue(configuration.getValidate()); + } +} diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java new file mode 100644 index 000000000000..e4eda0d7520b --- /dev/null +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java @@ -0,0 +1,278 @@ +/* + * 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.beam.sdk.io.snowflake.test.unit.read; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.io.AvroGeneratedUser; +import org.apache.beam.sdk.io.snowflake.SnowflakeIO; +import org.apache.beam.sdk.io.snowflake.SnowflakeService; +import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeBasicDataSource; +import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeDatabase; +import org.apache.beam.sdk.io.snowflake.test.FakeSnowflakeServiceImpl; +import org.apache.beam.sdk.io.snowflake.test.unit.BatchTestPipelineOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SnowflakeIOReadTest implements Serializable { + public static final String FAKE_TABLE = "FAKE_TABLE"; + public static final String FAKE_QUERY = "SELECT * FROM FAKE_TABLE"; + + private static final BatchTestPipelineOptions options = + TestPipeline.testingPipelineOptions().as(BatchTestPipelineOptions.class);; + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + private static SnowflakeIO.DataSourceConfiguration dataSourceConfiguration; + private static SnowflakeService snowflakeService; + private static String stagingBucketName; + private static String integrationName; + private static List avroTestData; + + @BeforeClass + public static void setup() { + + List testData = Arrays.asList("Paul,51,red", "Jackson,41,green"); + + avroTestData = + ImmutableList.of( + new AvroGeneratedUser("Paul", 51, "red"), + new AvroGeneratedUser("Jackson", 41, "green")); + + FakeSnowflakeDatabase.createTableWithElements(FAKE_TABLE, testData); + + options.setServerName("NULL.snowflakecomputing.com"); + options.setStorageIntegration("STORAGE_INTEGRATION"); + options.setStagingBucketName("BUCKET"); + + stagingBucketName = options.getStagingBucketName(); + integrationName = options.getStorageIntegration(); + + dataSourceConfiguration = + SnowflakeIO.DataSourceConfiguration.create(new FakeSnowflakeBasicDataSource()) + .withServerName(options.getServerName()); + + snowflakeService = new FakeSnowflakeServiceImpl(); + } + + @Test + public void testConfigIsMissingStagingBucketName() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("withStagingBucketName() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable(FAKE_TABLE) + .withIntegrationName(integrationName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testConfigIsMissingIntegrationName() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("withIntegrationName() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testConfigIsMissingCsvMapper() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("withCsvMapper() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testConfigIsMissingCoder() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("withCoder() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper())); + + pipeline.run(); + } + + @Test + public void testConfigIsMissingFromTableOrFromQuery() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("fromTable() or fromQuery() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testConfigIsMissingDataSourceConfiguration() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("withDataSourceConfiguration() or withDataSourceProviderFn() is required"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testConfigContainsFromQueryAndFromTable() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("fromTable() and fromQuery() are not allowed together"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromQuery("") + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testTableDoesntExist() { + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage("SQL compilation error: Table does not exist"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable("NON_EXIST") + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testInvalidQuery() { + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage("SQL compilation error: Invalid query"); + + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromQuery("BAD_QUERY") + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + pipeline.run(); + } + + @Test + public void testReadFromTable() { + PCollection items = + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromTable(FAKE_TABLE) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + PAssert.that(items).containsInAnyOrder(avroTestData); + pipeline.run(); + } + + @Test + public void testReadFromQuery() { + PCollection items = + pipeline.apply( + SnowflakeIO.read(snowflakeService) + .withDataSourceConfiguration(dataSourceConfiguration) + .fromQuery(FAKE_QUERY) + .withStagingBucketName(stagingBucketName) + .withIntegrationName(integrationName) + .withCsvMapper(getCsvMapper()) + .withCoder(AvroCoder.of(AvroGeneratedUser.getClassSchema()))); + + PAssert.that(items).containsInAnyOrder(avroTestData); + pipeline.run(); + } + + static SnowflakeIO.CsvMapper getCsvMapper() { + return (SnowflakeIO.CsvMapper) + parts -> + new GenericRecordBuilder(AvroGeneratedUser.getClassSchema()) + .set("name", String.valueOf(parts[0])) + .set("favorite_number", Integer.valueOf(parts[1])) + .set("favorite_color", String.valueOf(parts[2])) + .build(); + } +} diff --git a/sdks/java/io/snowflake/src/test/resources/test_rsa_key.p8 b/sdks/java/io/snowflake/src/test/resources/test_rsa_key.p8 new file mode 100644 index 000000000000..ee86a8dd39dd --- /dev/null +++ b/sdks/java/io/snowflake/src/test/resources/test_rsa_key.p8 @@ -0,0 +1,29 @@ +-----BEGIN ENCRYPTED PRIVATE KEY----- +MIIE6TAbBgkqhkiG9w0BBQMwDgQI2sbwjlr6RrcCAggABIIEyPbY/oiy8EH1QY9t +EVlG2GyZK07bYsVP57PfFWVTJi6gN7G8zxF8vQvPN2fm8w1GJTzyz+ZAdxAXEv6/ +Oe0KmvYbe+YeO0+u+aZah8rnhFgAo1OgMDA8hCKLL98qrGau03TRGZKD6Xxce3nZ +DizFbCxcRejnJflWUFWgyro45Qnb9jtOop+rnmqHeV/CMP/RLattYsZ0SrcCnr1O +fIfb7jqNjY94V7xh9O8G1g5YHL3fv8ir3iXLpy7wPjMVHSMGcK092tnAK9/okmiA +EdzNH/DfwUQ2qZ4gWJJp75b5RwVTZ9uNDQZkQJQLAS8zowtjXhKJ+Zy26ID/EI7u +H5R5WhUl2ROI2ssjX8I0wFyOMA5guBjw2xL/Y+/eI3dQ+2g6hxN2TpPoKEAzeM1a +OyrTyyVZ7nJyJ7RC6odRnHE9PLFzpmGv/3YhbSvhAwAnLN2mky+LqSw8hTGVMaIv +QbnkeqxGGd6miIZgPOrbBj1mRErLdgKkJn9UNYPXKoB+0gEyLGYSkY1TdJ6NjRq8 +5oU+MgU1dnyp73JMiafz4AbFLHrxXG3AG6vww9WBgiV9pFmQSMcuVX8p8rvxJ8H4 +nFQlwiZrvl98xcfGICkoiSKP2fEDir1sdbjpcEY1Rxk3baUqBxZCrRKo+Dz4k4yy +VjZX8SkXrYgcQNxmDiv59D34QYtKyq8ZxeHB7tVrj8/G1N8WgU8EApR0+yQL+0TP +8aicPc++9ta62Wv59iqmsQeR+Bdq1/kOTZCMA9QMK7/mJpcn18/5EmbrAUbw/2pf +TdyLqEXUf3N0zrDUAKUuWSOJMHOEhdcvQwVfyj3zy8O2+aM2PYos3c5pKQ05X/RJ +6bPl6taxcEsHQByNC0+7JJ1yPxYlqW28uDao4XNkrwhSkhzv0DylAGZNIrwAFkKq +dXRdCBMijvBB7jKvfPKK0aOlVJ7fdRo6PPoAJaDhmfsd5lbIGFcpwu4Rf2AiJQq3 +7ZzfGSUH/uvTsXW/e+QOQkr8cI4apRjGAuTImoIebZgkU8U9NOjCIwtBZO+KgiRZ +4cjC4mXgxMAdmmIMXUqy0QGiqgK2IGhpDneC9y2Al1WT/7Sz5au6tvMEIf5yrYuW +62/LyErTfNiwFGa4gYQ0nQ21ifwrA5bFFCyf4K0NyFRZaKJd/gpnszZtuw9NKxLn +5Lz10bI/aFPcOyKxgX0DqYHhiJFw0v4uTUIQK7RkDCQ+xAiJJ7c53/piCaqc6+IY +BRHvKTRK2jIxSlL8In+MxL+hDPXBm1c/NLIAqMogh7u0qYRg3U0V9leH5vZqatHI +/SDPyMAbcrjuIcg04fjaH2KW/REdPL87heoqH0tH5x0PnQqAUGxmuUm/7FEcoB+Q +oQOD/KIkZ5Abenmw/VJW11tt9A5dV6d3y+OBaTN2U1ZzT8PaWchjUimsIY3CVxTM +h5IxI1VTqMy9o/5mkA5ishzdUxh0hReO9NzUx4zgKFuWHAnUqYEkGC1okdSm+DqM +s3jtYwZcbXhV8USCZJWEyfV7T5/1iXR2/U432e7HN6Wv1uC/GQWafelkKosr2ulG +Y9Heehs56te3osz62G8Y27gCdZGi+GnysgegiLg7E2Qaep3UGk+Q3h8E+YAyQ0eK +H8gI6sKRjdIAGuhs7w== +-----END ENCRYPTED PRIVATE KEY----- diff --git a/settings.gradle b/settings.gradle index 0c642309dfed..8a80ffd06298 100644 --- a/settings.gradle +++ b/settings.gradle @@ -119,6 +119,7 @@ include ":sdks:java:io:parquet" include ":sdks:java:io:rabbitmq" include ":sdks:java:io:redis" include ":sdks:java:io:solr" +include ":sdks:java:io:snowflake" include ":sdks:java:io:thrift" include ":sdks:java:io:tika" include ":sdks:java:io:xml" From 9f8783d4d1d759d6c37fe4bfa1113723a4f5d37b Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 21 May 2020 17:19:22 -0700 Subject: [PATCH 052/151] add def args --- release/src/main/groovy/MobileGamingCommands.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/release/src/main/groovy/MobileGamingCommands.groovy b/release/src/main/groovy/MobileGamingCommands.groovy index 505fbcf691c0..1042062e4cb5 100644 --- a/release/src/main/groovy/MobileGamingCommands.groovy +++ b/release/src/main/groovy/MobileGamingCommands.groovy @@ -146,7 +146,7 @@ class MobileGamingCommands { } private Map getLeaderBoardWithStreamingEngineArgs(String runner, String jobName){ - args = [project: testScripts.gcpProject(), + def args = [project: testScripts.gcpProject(), dataset: testScripts.bqDataset(), topic: "projects/${testScripts.gcpProject()}/topics/${testScripts.pubsubTopic()}", leaderBoardTableName: "leaderboard_${runner}", @@ -160,7 +160,7 @@ class MobileGamingCommands { } private Map getGameStatsArgs(String runner, String jobName){ - args = [project: testScripts.gcpProject(), + def args = [project: testScripts.gcpProject(), dataset: testScripts.bqDataset(), topic: "projects/${testScripts.gcpProject()}/topics/${testScripts.pubsubTopic()}", fixedWindowDuration: 5, From 8f53b99c8567ab25f51cb85239a2e3fe70fa0898 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Thu, 21 May 2020 22:16:45 -0400 Subject: [PATCH 053/151] Fix typo Portabillity -> Portability --- website/www/site/content/en/roadmap/go-sdk.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/roadmap/go-sdk.md b/website/www/site/content/en/roadmap/go-sdk.md index 8e003e689cca..4172cb09cb59 100644 --- a/website/www/site/content/en/roadmap/go-sdk.md +++ b/website/www/site/content/en/roadmap/go-sdk.md @@ -18,7 +18,7 @@ limitations under the License. # Go SDK Roadmap The Go SDK is currently experimental. As the first purely portable Beam SDK, the Go SDK is constrained -by the status of the [Beam Portabillity Framework](https://beam.apache.org/roadmap/portability/) and the existence of +by the status of the [Beam Portability Framework](https://beam.apache.org/roadmap/portability/) and the existence of portable runners. **April 2020 Update** From 1095db056dbd9095c35effbff07cbcd90dc87c54 Mon Sep 17 00:00:00 2001 From: Daniel Oliveira Date: Wed, 20 May 2020 12:06:49 -0700 Subject: [PATCH 054/151] [BEAM-9978] Adding functionality and tests to Go offset range tracker. Pretty simple. Moves some commonly desired behaviors out of the SDF code and into the offset range tracker/restriction code + adds tests. --- sdks/go/examples/stringsplit/stringsplit.go | 16 +- .../io/rtrackers/offsetrange/offsetrange.go | 63 +++++- .../rtrackers/offsetrange/offsetrange_test.go | 212 ++++++++++++++++++ sdks/go/pkg/beam/io/synthetic/source.go | 31 +-- sdks/go/pkg/beam/io/synthetic/step.go | 31 +-- 5 files changed, 276 insertions(+), 77 deletions(-) create mode 100644 sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange_test.go diff --git a/sdks/go/examples/stringsplit/stringsplit.go b/sdks/go/examples/stringsplit/stringsplit.go index eff55bf90cc6..d96a3417ac08 100644 --- a/sdks/go/examples/stringsplit/stringsplit.go +++ b/sdks/go/examples/stringsplit/stringsplit.go @@ -71,19 +71,7 @@ func (fn *StringSplitFn) CreateInitialRestriction(s string) offsetrange.Restrict // SplitRestriction performs initial splits so that each restriction is split // into 5. func (fn *StringSplitFn) SplitRestriction(s string, rest offsetrange.Restriction) []offsetrange.Restriction { - size := rest.End - rest.Start - splitPts := []int64{ - rest.Start, - rest.Start + (size / 5), - rest.Start + (size * 2 / 5), - rest.Start + (size * 3 / 5), - rest.Start + (size * 4 / 5), - rest.End, - } - var splits []offsetrange.Restriction - for i := 0; i < len(splitPts)-1; i++ { - splits = append(splits, offsetrange.Restriction{Start: splitPts[i], End: splitPts[i+1]}) - } + splits := rest.EvenSplits(5) log.Debugf(context.Background(), "StringSplit SplitRestrictions: %v -> %v", rest, splits) return splits } @@ -91,7 +79,7 @@ func (fn *StringSplitFn) SplitRestriction(s string, rest offsetrange.Restriction // RestrictionSize returns the size as the difference between the restriction's // start and end. func (fn *StringSplitFn) RestrictionSize(s string, rest offsetrange.Restriction) float64 { - size := float64(rest.End - rest.Start) + size := rest.Size() log.Debugf(context.Background(), "StringSplit RestrictionSize: %v -> %v", rest, size) return size } diff --git a/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go index 5e55f35ea592..fb1e5965431a 100644 --- a/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go +++ b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange.go @@ -21,6 +21,7 @@ package offsetrange import ( "errors" + "math" "reflect" "github.com/apache/beam/sdks/go/pkg/beam" @@ -31,8 +32,43 @@ func init() { beam.RegisterType(reflect.TypeOf((*Restriction)(nil))) } +// Restriction is an offset range restriction, which represents a range of +// integers as a half-closed interval with boundaries [start, end). type Restriction struct { - Start, End int64 // Half-closed interval with boundaries [start, end). + Start, End int64 +} + +// EvenSplits splits a restriction into a number of evenly sized restrictions. +// Each split restriction is guaranteed to not be empty, and each unit from the +// original restriction is guaranteed to be contained in one split restriction. +// +// Num should be greater than 0. Otherwise there is no way to split the +// restriction and this function will return the original restriction. +func (r *Restriction) EvenSplits(num int64) (splits []Restriction) { + if num <= 1 { + // Don't split, just return original restriction. + return append(splits, *r) + } + + offset := r.Start + size := r.End - r.Start + for i := int64(0); i < num; i++ { + split := Restriction{ + Start: offset + (i * size / num), + End: offset + ((i + 1) * size / num), + } + // Skip restrictions that end up empty. + if split.End-split.Start <= 0 { + continue + } + splits = append(splits, split) + } + return splits +} + +// Size returns the restriction's size as the difference between Start and End. +func (r *Restriction) Size() float64 { + return float64(r.End - r.Start) } // Tracker tracks a restriction that can be represented as a range of integer values, @@ -42,9 +78,8 @@ type Restriction struct { type Tracker struct { Rest Restriction Claimed int64 // Tracks the last claimed position. - Stopped bool // Tracks whether TryClaim has already indicated to stop processing elements for - // any reason. - Err error + Stopped bool // Tracks whether TryClaim has indicated to stop processing elements. + Err error } // NewTracker is a constructor for an Tracker given a start and end range. @@ -57,10 +92,15 @@ func NewTracker(rest Restriction) *Tracker { } } -// TryClaim accepts an int64 position and successfully claims it if that position is greater than -// the previously claimed position and less than the end of the restriction. Note that the -// Tracker is not considered done until a position >= tracker.end tries to be claimed, -// at which point this method signals to end processing. +// TryClaim accepts an int64 position representing the starting position of a block of work. It +// successfully claims it if the position is greater than the previously claimed position and within +// the restriction. Claiming a position at or beyond the end of the restriction signals that the +// entire restriction has been processed and is now done, at which point this method signals to end +// processing. +// +// The tracker stops with an error if a claim is attempted after the tracker has signalled to stop, +// if a position is claimed before the start of the restriction, or if a position is claimed before +// the latest successfully claimed. func (tracker *Tracker) TryClaim(rawPos interface{}) bool { if tracker.Stopped == true { tracker.Err = errors.New("cannot claim work after restriction tracker returns false") @@ -88,7 +128,7 @@ func (tracker *Tracker) TryClaim(rawPos interface{}) bool { return true } -// IsDone returns true if the most recent claimed element is past the end of the restriction. +// GetError returns the error that caused the tracker to stop, if there is one. func (tracker *Tracker) GetError() error { return tracker.Err } @@ -105,7 +145,8 @@ func (tracker *Tracker) TrySplit(fraction float64) (primary, residual interface{ fraction = 1 } - splitPt := tracker.Claimed + int64(fraction*float64(tracker.Rest.End-tracker.Claimed)) + // Use Ceil to always round up from float split point. + splitPt := tracker.Claimed + int64(math.Ceil(fraction*float64(tracker.Rest.End-tracker.Claimed))) if splitPt >= tracker.Rest.End { return tracker.Rest, nil, nil } @@ -123,5 +164,5 @@ func (tracker *Tracker) GetProgress() (done, remaining float64) { // IsDone returns true if the most recent claimed element is past the end of the restriction. func (tracker *Tracker) IsDone() bool { - return tracker.Claimed >= tracker.Rest.End + return tracker.Err == nil && tracker.Claimed >= tracker.Rest.End } diff --git a/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange_test.go b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange_test.go new file mode 100644 index 000000000000..489e25146249 --- /dev/null +++ b/sdks/go/pkg/beam/io/rtrackers/offsetrange/offsetrange_test.go @@ -0,0 +1,212 @@ +// 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 offsetrange + +import ( + "fmt" + "github.com/google/go-cmp/cmp" + "testing" +) + +// TestRestriction_EvenSplits tests various splits and checks that they all +// follow the contract for EvenSplits. This means that all restrictions are +// evenly split, that each restriction has at least one element, and that each +// element is present in the split restrictions. +func TestRestriction_EvenSplits(t *testing.T) { + tests := []struct { + rest Restriction + num int64 + }{ + {rest: Restriction{Start: 0, End: 21}, num: 4}, + {rest: Restriction{Start: 21, End: 42}, num: 4}, + {rest: Restriction{Start: 0, End: 5}, num: 10}, + {rest: Restriction{Start: 0, End: 21}, num: -1}, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(rest[%v, %v], splits = %v)", + test.rest.Start, test.rest.End, test.num), func(t *testing.T) { + r := test.rest + + // Get the minimum size that a split restriction can be. Max size + // should be min + 1. This way we can check the size of each split. + num := test.num + if num <= 1 { + num = 1 + } + min := (r.End - r.Start) / num + + splits := r.EvenSplits(test.num) + prevEnd := r.Start + for _, split := range splits { + size := split.End - split.Start + // Check: Each restriction has at least 1 element. + if size == 0 { + t.Errorf("split restriction [%v, %v] is empty, size must be greater than 0.", + split.Start, split.End) + } + // Check: Restrictions are evenly split. + if size != min && size != min+1 { + t.Errorf("split restriction [%v, %v] has unexpected size. got: %v, want: %v or %v", + split.Start, split.End, size, min, min+1) + } + // Check: All elements are still in a split restrictions. This + // logic assumes that the splits are returned in order which + // isn't guaranteed by EvenSplits, but this check is way easier + // with the assumption. + if split.Start != prevEnd { + t.Errorf("restriction range [%v, %v] missing after splits.", + prevEnd, split.Start) + } else { + prevEnd = split.End + } + } + if prevEnd != r.End { + t.Errorf("restriction range [%v, %v] missing after splits.", + prevEnd, r.End) + } + }) + } +} + +// TestTracker_TryClaim validates both success and failure cases for TryClaim. +func TestTracker_TryClaim(t *testing.T) { + // Test that TryClaim works as expected when called correctly. + t.Run("Correctness", func(t *testing.T) { + tests := []struct { + rest Restriction + claims []int64 + }{ + {rest: Restriction{Start: 0, End: 3}, claims: []int64{0, 1, 2, 3}}, + {rest: Restriction{Start: 10, End: 40}, claims: []int64{15, 20, 50}}, + {rest: Restriction{Start: 0, End: 3}, claims: []int64{4}}, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(rest[%v, %v], claims = %v)", + test.rest.Start, test.rest.End, test.claims), func(t *testing.T) { + rt := NewTracker(test.rest) + for _, pos := range test.claims { + // If TryClaim returns false, check if there was an error. + if !rt.TryClaim(pos) && !rt.IsDone() { + t.Fatalf("tracker claiming %v failed, error: %v", pos, rt.GetError()) + } + } + }) + } + }) + + // Test that each invalid error case actually results in an error. + t.Run("Errors", func(t *testing.T) { + tests := []struct { + rest Restriction + claims []int64 + }{ + // Claiming backwards. + {rest: Restriction{Start: 0, End: 3}, claims: []int64{0, 2, 1}}, + // Claiming before start of restriction. + {rest: Restriction{Start: 10, End: 40}, claims: []int64{8}}, + // Claiming after tracker signalled to stop. + {rest: Restriction{Start: 0, End: 3}, claims: []int64{4, 5}}, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(rest[%v, %v], claims = %v)", + test.rest.Start, test.rest.End, test.claims), func(t *testing.T) { + rt := NewTracker(test.rest) + for _, pos := range test.claims { + // Finish successfully if we got an error. + if !rt.TryClaim(pos) && !rt.IsDone() && rt.GetError() != nil { + return + } + } + t.Fatal("tracker did not fail on invalid claim") + }) + } + }) +} + +// TestTracker_TrySplit tests that TrySplit follows its contract, meaning that +// splits don't lose any elements, split fractions are clamped to 0 or 1, and +// that TrySplit always splits at the nearest integer greater than the given +// fraction. +func TestTracker_TrySplit(t *testing.T) { + tests := []struct { + rest Restriction + claimed int64 + fraction float64 + // Index where we want the split to happen. This will be the end + // (exclusive) of the primary and first element of the residual. + splitPt int64 + }{ + { + rest: Restriction{Start: 0, End: 1}, + claimed: 0, + fraction: 0.5, + splitPt: 1, + }, + { + rest: Restriction{Start: 0, End: 5}, + claimed: 0, + fraction: 0.5, + splitPt: 3, + }, + { + rest: Restriction{Start: 0, End: 10}, + claimed: 5, + fraction: 0.5, + splitPt: 8, + }, + { + rest: Restriction{Start: 0, End: 10}, + claimed: 5, + fraction: -0.5, + splitPt: 5, + }, + { + rest: Restriction{Start: 0, End: 10}, + claimed: 5, + fraction: 1.5, + splitPt: 10, + }, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(split at %v of [%v, %v])", + test.fraction, test.claimed, test.rest.End), func(t *testing.T) { + rt := NewTracker(test.rest) + ok := rt.TryClaim(test.claimed) + if !ok { + t.Fatalf("tracker failed on initial claim: %v", test.claimed) + } + gotP, gotR, err := rt.TrySplit(test.fraction) + if err != nil { + t.Fatalf("tracker failed on split: %v", err) + } + var wantP interface{} = Restriction{Start: test.rest.Start, End: test.splitPt} + var wantR interface{} = Restriction{Start: test.splitPt, End: test.rest.End} + if test.splitPt == test.rest.End { + wantR = nil // When residuals are empty we should get nil. + } + if !cmp.Equal(gotP, wantP) { + t.Errorf("split got incorrect primary: got: %v, want: %v", gotP, wantP) + } + if !cmp.Equal(gotR, wantR) { + t.Errorf("split got incorrect residual: got: %v, want: %v", gotR, wantR) + } + }) + } +} diff --git a/sdks/go/pkg/beam/io/synthetic/source.go b/sdks/go/pkg/beam/io/synthetic/source.go index 00033206cf62..fbe385c6bd32 100644 --- a/sdks/go/pkg/beam/io/synthetic/source.go +++ b/sdks/go/pkg/beam/io/synthetic/source.go @@ -24,10 +24,11 @@ package synthetic import ( "fmt" - "github.com/apache/beam/sdks/go/pkg/beam" - "github.com/apache/beam/sdks/go/pkg/beam/io/rtrackers/offsetrange" "math/rand" "time" + + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/io/rtrackers/offsetrange" ) // Source creates a synthetic source transform that emits randomly @@ -95,35 +96,13 @@ func (fn *sourceFn) CreateInitialRestriction(config SourceConfig) offsetrange.Re // method will contain at least one element, so the number of splits will not // exceed the number of elements. func (fn *sourceFn) SplitRestriction(config SourceConfig, rest offsetrange.Restriction) (splits []offsetrange.Restriction) { - if config.InitialSplits <= 1 { - // Don't split, just return original restriction. - return append(splits, rest) - } - - // TODO(BEAM-9978) Move this implementation of the offset range restriction - // splitting to the restriction itself, and add testing. - num := int64(config.InitialSplits) - offset := rest.Start - size := rest.End - rest.Start - for i := int64(0); i < num; i++ { - split := offsetrange.Restriction{ - Start: offset + (i * size / num), - End: offset + ((i + 1) * size / num), - } - // Skip restrictions that end up empty. - if split.End-split.Start <= 0 { - continue - } - splits = append(splits, split) - } - return splits + return rest.EvenSplits(int64(config.InitialSplits)) } // RestrictionSize outputs the size of the restriction as the number of elements // that restriction will output. func (fn *sourceFn) RestrictionSize(config SourceConfig, rest offsetrange.Restriction) float64 { - // TODO(BEAM-9978) Move this size implementation to the offset range restriction itself. - return float64(rest.End - rest.Start) + return rest.Size() } // CreateTracker just creates an offset range restriction tracker for the diff --git a/sdks/go/pkg/beam/io/synthetic/step.go b/sdks/go/pkg/beam/io/synthetic/step.go index 977d5540e410..4fc23bc15d56 100644 --- a/sdks/go/pkg/beam/io/synthetic/step.go +++ b/sdks/go/pkg/beam/io/synthetic/step.go @@ -17,10 +17,11 @@ package synthetic import ( "fmt" - "github.com/apache/beam/sdks/go/pkg/beam" - "github.com/apache/beam/sdks/go/pkg/beam/io/rtrackers/offsetrange" "math/rand" "time" + + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/io/rtrackers/offsetrange" ) // Step creates a synthetic step transform that receives KV<[]byte, []byte> @@ -98,35 +99,13 @@ func (fn *sdfStepFn) CreateInitialRestriction(key, val []byte) offsetrange.Restr // method will contain at least one element, so the number of splits will not // exceed the number of elements. func (fn *sdfStepFn) SplitRestriction(key, val []byte, rest offsetrange.Restriction) (splits []offsetrange.Restriction) { - if fn.cfg.InitialSplits <= 1 { - // Don't split, just return original restriction. - return append(splits, rest) - } - - // TODO(BEAM-9978) Move this implementation of the offset range restriction - // splitting to the restriction itself, and add testing. - num := int64(fn.cfg.InitialSplits) - offset := rest.Start - size := rest.End - rest.Start - for i := int64(0); i < num; i++ { - split := offsetrange.Restriction{ - Start: offset + (i * size / num), - End: offset + ((i + 1) * size / num), - } - // Skip restrictions that end up empty. - if split.End-split.Start <= 0 { - continue - } - splits = append(splits, split) - } - return splits + return rest.EvenSplits(int64(fn.cfg.InitialSplits)) } // RestrictionSize outputs the size of the restriction as the number of elements // that restriction will output. func (fn *sdfStepFn) RestrictionSize(key, val []byte, rest offsetrange.Restriction) float64 { - // TODO(BEAM-9978) Move this size implementation to the offset range restriction itself. - return float64(rest.End - rest.Start) + return rest.Size() } // CreateTracker creates an offset range restriction tracker for the From 64f8a40c26afeae7fead09f06ed18b930ce1731b Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Thu, 21 May 2020 23:37:02 -0700 Subject: [PATCH 055/151] [BEAM-10064] Fix google3 import error for BEAM-9383 --- .../apache_beam/runners/dataflow/internal/apiclient_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py index 7776156bd460..7a780a6a5192 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py @@ -944,7 +944,8 @@ def test_stage_resources(self): '--temp_location', 'gs://test-location/temp', '--staging_location', - 'gs://test-location/staging' + 'gs://test-location/staging', + '--no_auth' ]) pipeline = beam_runner_api_pb2.Pipeline( components=beam_runner_api_pb2.Components( From 6766d0c2b929f74152c4d4a60bd0898c918ad667 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Fri, 22 May 2020 12:06:27 +0200 Subject: [PATCH 056/151] [BEAM-10058] Provide less strict assertion to make the test more resistant against future changes in a model --- .../apache_beam/ml/gcp/videointelligenceml_test_it.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/gcp/videointelligenceml_test_it.py b/sdks/python/apache_beam/ml/gcp/videointelligenceml_test_it.py index f411549d7b9c..397256e5f153 100644 --- a/sdks/python/apache_beam/ml/gcp/videointelligenceml_test_it.py +++ b/sdks/python/apache_beam/ml/gcp/videointelligenceml_test_it.py @@ -44,7 +44,7 @@ def extract_entities_descriptions(response): for result in response.annotation_results: - for segment in result.segment_label_annotations: + for segment in result.segment_presence_label_annotations: yield segment.entity.description @@ -69,7 +69,9 @@ def test_label_detection_with_video_context(self): | beam.ParDo(extract_entities_descriptions) | beam.combiners.ToList()) - assert_that(output, matches_all([hc.has_items('bicycle', 'dinosaur')])) + # Search for at least one entity that contains 'bicycle'. + assert_that( + output, matches_all([hc.has_item(hc.contains_string('bicycle'))])) if __name__ == '__main__': From 790ed4427429292ae14931ca5c3ad9af73438e94 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 10:36:42 -0400 Subject: [PATCH 057/151] Fix typo copyLicenseScrips -> copyLicenseScripts --- sdks/python/container/py2/build.gradle | 4 ++-- sdks/python/container/py35/build.gradle | 4 ++-- sdks/python/container/py36/build.gradle | 4 ++-- sdks/python/container/py37/build.gradle | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/python/container/py2/build.gradle b/sdks/python/container/py2/build.gradle index 4e5848208fa0..2821600419d7 100644 --- a/sdks/python/container/py2/build.gradle +++ b/sdks/python/container/py2/build.gradle @@ -43,7 +43,7 @@ task copyDockerfileDependencies(type: Copy) { } } -task copyLicenseScrips(type: Copy){ +task copyLicenseScripts(type: Copy){ from ("../license_scripts") into "build/target/license_scripts" } @@ -72,4 +72,4 @@ docker { dockerPrepare.dependsOn copyLauncherDependencies dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScrips +dockerPrepare.dependsOn copyLicenseScripts diff --git a/sdks/python/container/py35/build.gradle b/sdks/python/container/py35/build.gradle index ce1fbe91b16a..6a0133dd9916 100644 --- a/sdks/python/container/py35/build.gradle +++ b/sdks/python/container/py35/build.gradle @@ -43,7 +43,7 @@ task copyDockerfileDependencies(type: Copy) { } } -task copyLicenseScrips(type: Copy){ +task copyLicenseScripts(type: Copy){ from ("../license_scripts") into "build/target/license_scripts" } @@ -72,4 +72,4 @@ docker { dockerPrepare.dependsOn copyLauncherDependencies dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScrips +dockerPrepare.dependsOn copyLicenseScripts diff --git a/sdks/python/container/py36/build.gradle b/sdks/python/container/py36/build.gradle index c5417169c76b..b50267c29201 100644 --- a/sdks/python/container/py36/build.gradle +++ b/sdks/python/container/py36/build.gradle @@ -43,7 +43,7 @@ task copyDockerfileDependencies(type: Copy) { } } -task copyLicenseScrips(type: Copy){ +task copyLicenseScripts(type: Copy){ from ("../license_scripts") into "build/target/license_scripts" } @@ -72,4 +72,4 @@ docker { dockerPrepare.dependsOn copyLauncherDependencies dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScrips +dockerPrepare.dependsOn copyLicenseScripts diff --git a/sdks/python/container/py37/build.gradle b/sdks/python/container/py37/build.gradle index 1225332d9535..969a58db7945 100644 --- a/sdks/python/container/py37/build.gradle +++ b/sdks/python/container/py37/build.gradle @@ -43,7 +43,7 @@ task copyDockerfileDependencies(type: Copy) { } } -task copyLicenseScrips(type: Copy){ +task copyLicenseScripts(type: Copy){ from ("../license_scripts") into "build/target/license_scripts" } @@ -72,4 +72,4 @@ docker { dockerPrepare.dependsOn copyLauncherDependencies dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScrips +dockerPrepare.dependsOn copyLicenseScripts From 0e484983febc9ed6be0ddf5b68452380e208db56 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 11:06:58 -0400 Subject: [PATCH 058/151] [BEAM-10067] Minify website assets with --minify flag --- website/www/package.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/package.json b/website/www/package.json index a5b1cbea637d..d68017f163e1 100644 --- a/website/www/package.json +++ b/website/www/package.json @@ -7,7 +7,7 @@ "scripts": { "build_github_samples": "./build_github_samples.sh", "develop": "cd site && hugo server", - "build": "cross-env HUGO_ENV=production hugo -d ../dist -s site -v", + "build": "cross-env HUGO_ENV=production hugo --minify -d ../dist -s site -v", "start": "hugo -d ../dist -s site -vw" }, "dependencies": {}, From e45623e5d7f72b28c0ec1bde1c1795982cbe9ffe Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 22 May 2020 10:01:42 -0700 Subject: [PATCH 059/151] [BEAM-10051] Move closed reader check after sentinel. (#11768) --- .../pkg/beam/core/runtime/harness/datamgr.go | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 81e813e4f93b..3fb9401dd017 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -266,18 +266,14 @@ func (c *DataChannel) read(ctx context.Context) { cache[id] = r } - if r.completed { - // The local reader has closed but the remote is still sending data. - // Just ignore it. We keep the reader config in the cache so we don't - // treat it as a new reader. Eventually the stream will finish and go - // through normal teardown. - continue - } // TODO(BEAM-9558): Cleanup once dataflow is updated. if len(elm.GetData()) == 0 || elm.GetIsLast() { - // Sentinel EOF segment for stream. Close buffer to signal EOF. - r.completed = true - close(r.buf) + // If this reader hasn't closed yet, do so now. + if !r.completed { + // Sentinel EOF segment for stream. Close buffer to signal EOF. + r.completed = true + close(r.buf) + } // Clean up local bookkeeping. We'll never see another message // for it again. We have to be careful not to remove the real @@ -287,6 +283,14 @@ func (c *DataChannel) read(ctx context.Context) { continue } + if r.completed { + // The local reader has closed but the remote is still sending data. + // Just ignore it. We keep the reader config in the cache so we don't + // treat it as a new reader. Eventually the stream will finish and go + // through normal teardown. + continue + } + // This send is deliberately blocking, if we exceed the buffering for // a reader. We can't buffer the entire main input, if some user code // is slow (or gets stuck). If the local side closes, the reader From 455fca5ab79f74d4e79a81e66942878f35aaa08c Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 22 May 2020 10:02:42 -0700 Subject: [PATCH 060/151] [BEAM-10056] Fix validation for struct CoGBKs (#11782) --- sdks/go/pkg/beam/core/graph/fn.go | 24 ++++--- sdks/go/pkg/beam/core/graph/fn_test.go | 91 +++++++++++++++++++++----- sdks/go/pkg/beam/pardo.go | 11 ++-- 3 files changed, 97 insertions(+), 29 deletions(-) diff --git a/sdks/go/pkg/beam/core/graph/fn.go b/sdks/go/pkg/beam/core/graph/fn.go index 6b4f64bd03e8..1711d7cf7db8 100644 --- a/sdks/go/pkg/beam/core/graph/fn.go +++ b/sdks/go/pkg/beam/core/graph/fn.go @@ -297,6 +297,19 @@ func NumMainInputs(num mainInputs) func(*config) { } } +// CoGBKMainInput is an optional config to NewDoFn which specifies the number +// of components of a CoGBK input to the DoFn being created, allowing for more complete +// validation. +// +// Example usage: +// var col beam.PCollection +// graph.NewDoFn(fn, graph.CoGBKMainInput(len(col.Type().Components()))) +func CoGBKMainInput(components int) func(*config) { + return func(cfg *config) { + cfg.numMainIn = mainInputs(components) + } +} + // NewDoFn constructs a DoFn from the given value, if possible. func NewDoFn(fn interface{}, options ...func(*config)) (*DoFn, error) { ret, err := NewFn(fn) @@ -586,14 +599,9 @@ func validateSideInputsNumUnknown(processFnInputs []funcx.FnParam, method *funcx // Handle cases where method has no inputs. if !ok { - if numProcessIn <= int(MainKv) { - return nil // We're good, possible for there to be no side inputs. - } - err := errors.Errorf("side inputs expected in method %v", methodName) - return errors.SetTopLevelMsgf(err, - "Missing side inputs in the %v method of a DoFn. "+ - "If side inputs are present in %v those side inputs must also be present in %v.", - methodName, processElementName, methodName) + // If there's no inputs, this is fine, as the ProcessElement method could be a + // CoGBK, and not have side inputs. + return nil } // Error if number of side inputs doesn't match any of the possible numbers of side inputs, diff --git a/sdks/go/pkg/beam/core/graph/fn_test.go b/sdks/go/pkg/beam/core/graph/fn_test.go index e5305123a4ed..f75410970fc7 100644 --- a/sdks/go/pkg/beam/core/graph/fn_test.go +++ b/sdks/go/pkg/beam/core/graph/fn_test.go @@ -26,22 +26,26 @@ import ( func TestNewDoFn(t *testing.T) { t.Run("valid", func(t *testing.T) { tests := []struct { - dfn interface{} - main mainInputs + dfn interface{} + opt func(*config) }{ - {dfn: func(string) int { return 0 }, main: MainSingle}, - {dfn: func(string, int) int { return 0 }, main: MainKv}, + {dfn: func(string) int { return 0 }, opt: NumMainInputs(MainSingle)}, + {dfn: func(string, int) int { return 0 }, opt: NumMainInputs(MainKv)}, {dfn: func(context.Context, typex.Window, typex.EventTime, reflect.Type, string, int, func(*int) bool, func() func(*int) bool, func(int)) (typex.EventTime, int, error) { return 0, 0, nil - }, main: MainKv}, - {dfn: &GoodDoFn{}, main: MainSingle}, - {dfn: &GoodDoFnOmittedMethods{}, main: MainSingle}, - {dfn: &GoodDoFnEmits{}, main: MainSingle}, - {dfn: &GoodDoFnSideInputs{}, main: MainSingle}, - {dfn: &GoodDoFnKv{}, main: MainKv}, - {dfn: &GoodDoFnKvSideInputs{}, main: MainKv}, - {dfn: &GoodDoFnAllExtras{}, main: MainKv}, - {dfn: &GoodDoFnUnexportedExtraMethod{}, main: MainSingle}, + }, opt: NumMainInputs(MainKv)}, + {dfn: &GoodDoFn{}, opt: NumMainInputs(MainSingle)}, + {dfn: &GoodDoFnOmittedMethods{}, opt: NumMainInputs(MainSingle)}, + {dfn: &GoodDoFnEmits{}, opt: NumMainInputs(MainSingle)}, + {dfn: &GoodDoFnSideInputs{}, opt: NumMainInputs(MainSingle)}, + {dfn: &GoodDoFnKv{}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodDoFnKvSideInputs{}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodDoFnAllExtras{}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodDoFnUnexportedExtraMethod{}, opt: NumMainInputs(MainSingle)}, + {dfn: &GoodDoFnCoGbk1{}, opt: NumMainInputs(MainKv)}, + {dfn: &GoodDoFnCoGbk2{}, opt: CoGBKMainInput(3)}, + {dfn: &GoodDoFnCoGbk7{}, opt: CoGBKMainInput(8)}, + {dfn: &GoodDoFnCoGbk1wSide{}, opt: NumMainInputs(MainKv)}, } for _, test := range tests { @@ -50,8 +54,10 @@ func TestNewDoFn(t *testing.T) { if _, err := NewDoFn(test.dfn); err != nil { t.Fatalf("NewDoFn failed: %v", err) } - if _, err := NewDoFn(test.dfn, NumMainInputs(test.main)); err != nil { - t.Fatalf("NewDoFn(NumMainInputs(%v)) failed: %v", test.main, err) + if _, err := NewDoFn(test.dfn, test.opt); err != nil { + cfg := defaultConfig() + test.opt(cfg) + t.Fatalf("NewDoFn(%#v) failed: %v", cfg, err) } }) } @@ -72,10 +78,8 @@ func TestNewDoFn(t *testing.T) { {dfn: &BadDoFnMismatchedEmitsStartBundle{}}, {dfn: &BadDoFnNoEmitsFinishBundle{}}, // Validate side inputs. - {dfn: &BadDoFnNoSideInputsStartBundle{}}, {dfn: &BadDoFnMissingSideInputsStartBundle{}}, {dfn: &BadDoFnMismatchedSideInputsStartBundle{}}, - {dfn: &BadDoFnNoSideInputsFinishBundle{}}, // Validate setup/teardown. {dfn: &BadDoFnParamsInSetup{}}, {dfn: &BadDoFnParamsInTeardown{}}, @@ -121,6 +125,11 @@ func TestNewDoFn(t *testing.T) { }, main: MainKv}, {dfn: &BadDoFnAmbiguousMainInput{}, main: MainKv}, {dfn: &BadDoFnAmbiguousSideInput{}, main: MainSingle}, + // These are ambiguous with CoGBKs, but should fail with known MainInputs. + {dfn: &BadDoFnNoSideInputsStartBundle{}, main: MainSingle}, + {dfn: &BadDoFnNoSideInputsStartBundle{}, main: MainKv}, + {dfn: &BadDoFnNoSideInputsFinishBundle{}, main: MainSingle}, + {dfn: &BadDoFnNoSideInputsFinishBundle{}, main: MainKv}, } for _, test := range tests { t.Run(reflect.TypeOf(test.dfn).String(), func(t *testing.T) { @@ -368,6 +377,54 @@ func (fn *GoodDoFnKvSideInputs) StartBundle(string, func(*int) bool, func() func func (fn *GoodDoFnKvSideInputs) FinishBundle(string, func(*int) bool, func() func(*int) bool) { } +type GoodDoFnCoGbk1 struct{} + +func (fn *GoodDoFnCoGbk1) ProcessElement(int, func(*string) bool) int { + return 0 +} + +func (fn *GoodDoFnCoGbk1) StartBundle() { +} + +func (fn *GoodDoFnCoGbk1) FinishBundle() { +} + +type GoodDoFnCoGbk2 struct{} + +func (fn *GoodDoFnCoGbk2) ProcessElement(int, func(*int) bool, func(*string) bool) int { + return 0 +} + +func (fn *GoodDoFnCoGbk2) StartBundle() { +} + +func (fn *GoodDoFnCoGbk2) FinishBundle() { +} + +type GoodDoFnCoGbk7 struct{} + +func (fn *GoodDoFnCoGbk7) ProcessElement(k int, v1, v2, v3, v4, v5, v6, v7 func(*int) bool) int { + return 0 +} + +func (fn *GoodDoFnCoGbk7) StartBundle() { +} + +func (fn *GoodDoFnCoGbk7) FinishBundle() { +} + +type GoodDoFnCoGbk1wSide struct{} + +func (fn *GoodDoFnCoGbk1wSide) ProcessElement(int, func(*string) bool, func(*int) bool) int { + return 0 +} + +func (fn *GoodDoFnCoGbk1wSide) StartBundle(func(*int) bool) { +} + +func (fn *GoodDoFnCoGbk1wSide) FinishBundle(func(*int) bool) { +} + type GoodDoFnAllExtras struct{} func (fn *GoodDoFnAllExtras) ProcessElement(context.Context, typex.Window, typex.EventTime, reflect.Type, string, int, func(*int) bool, func() func(*int) bool, func(int)) (typex.EventTime, int, error) { diff --git a/sdks/go/pkg/beam/pardo.go b/sdks/go/pkg/beam/pardo.go index d53e270e6a35..de3063446c79 100644 --- a/sdks/go/pkg/beam/pardo.go +++ b/sdks/go/pkg/beam/pardo.go @@ -17,6 +17,7 @@ package beam import ( "fmt" + "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder" "github.com/apache/beam/sdks/go/pkg/beam/core/typex" @@ -37,12 +38,14 @@ func TryParDo(s Scope, dofn interface{}, col PCollection, opts ...Option) ([]PCo return nil, addParDoCtx(err, s) } - num := graph.MainSingle + doFnOpt := graph.NumMainInputs(graph.MainSingle) // Check the PCollection for any keyed type (not just KV specifically). - if typex.IsKV(col.Type()) || typex.IsCoGBK(col.Type()) { - num = graph.MainKv + if typex.IsKV(col.Type()) { + doFnOpt = graph.NumMainInputs(graph.MainKv) + } else if typex.IsCoGBK(col.Type()) { + doFnOpt = graph.CoGBKMainInput(len(col.Type().Components())) } - fn, err := graph.NewDoFn(dofn, graph.NumMainInputs(num)) + fn, err := graph.NewDoFn(dofn, doFnOpt) if err != nil { return nil, addParDoCtx(err, s) } From f387df7d01ab1355f7f522ee20233b7674222b10 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 22:16:32 -0400 Subject: [PATCH 061/151] add io-matrix --- .../content/en/documentation/io/built-in.md | 82 +--- website/www/site/data/io_matrix.yaml | 373 ++++++++++++++++++ .../site/layouts/shortcodes/io-matrix.html | 48 +++ 3 files changed, 424 insertions(+), 79 deletions(-) create mode 100644 website/www/site/data/io_matrix.yaml create mode 100644 website/www/site/layouts/shortcodes/io-matrix.html diff --git a/website/www/site/content/en/documentation/io/built-in.md b/website/www/site/content/en/documentation/io/built-in.md index 481c532327f9..ba72908a885f 100644 --- a/website/www/site/content/en/documentation/io/built-in.md +++ b/website/www/site/content/en/documentation/io/built-in.md @@ -19,87 +19,11 @@ limitations under the License. This table contains the currently available I/O transforms. -Consult the [Programming Guide I/O section](/documentation/programming-guide#pipeline-io) for general usage instructions, and see the javadoc/pydoc for the particular I/O transforms. +Consult the [Programming Guide I/O section](/documentation/programming-guide#pipeline-io) for general usage instructions. +{{< language-switcher java py go >}} - - - - - - - - - - - - - - - - - - - - - - - - - -
    LanguageFile-basedMessagingDatabase
    Java -

    Beam Java supports Apache HDFS, Amazon S3, Google Cloud Storage, and local filesystems.

    -

    FileIO (general-purpose reading, writing, and matching of files)

    -

    AvroIO

    -

    TextIO

    -

    TFRecordIO

    -

    XmlIO

    -

    TikaIO

    -

    ParquetIO

    -

    ThriftIO

    -
    -

    Amazon Kinesis

    -

    AMQP

    -

    Apache Kafka

    -

    Google Cloud Pub/Sub

    -

    JMS

    -

    MQTT

    -

    RabbitMqIO

    -

    SqsIO

    -
    -

    Apache Cassandra

    -

    Apache Hadoop Input/Output Format

    -

    Apache HBase

    -

    Apache Hive (HCatalog)

    -

    Apache Kudu

    -

    Apache Solr

    -

    Elasticsearch (v2.x, v5.x, v6.x)

    -

    Google BigQuery

    -

    Google Cloud Bigtable

    -

    Google Cloud Datastore

    -

    Google Cloud Spanner

    -

    JDBC

    -

    MongoDB

    -

    Redis

    -
    Python/Batch -

    Beam Python supports Apache HDFS, Amazon S3, Google Cloud Storage, and local filesystems.

    -

    avroio

    -

    parquetio.py

    -

    textio

    -

    tfrecordio

    -

    vcfio

    -
    - -

    Google BigQuery

    -

    Google Cloud Datastore

    -

    Google Cloud Bigtable (Write)

    -

    MongoDB

    -
    Python/Streaming - -

    Google Cloud Pub/Sub

    -
    -

    Google BigQuery (sink only)

    -
    +{{< io-matrix >}} # In-Progress I/O Transforms diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml new file mode 100644 index 000000000000..4b64189ab730 --- /dev/null +++ b/website/www/site/data/io_matrix.yaml @@ -0,0 +1,373 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +categories: + - name: File-based + description: These I/O connectors involve working with files. + rows: + - transform: FileIO + description: "General-purpose transforms for working with files: listing files (matching), reading and writing." + implementations: + - language: java + name: org.apache.beam.sdk.io.FileIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/FileIO.html + - language: py + name: apache_beam.io.FileIO + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html + - transform: AvroIO + description: PTransforms for reading from and writing to Avro files. + implementations: + - language: java + name: org.apache.beam.sdk.io.AvroIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/AvroIO.html + - language: py + name: apache_beam.io.avroio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/avroio + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/avroio + - transform: TextIO + description: PTransforms for reading and writing text files. + implementations: + - language: java + name: org.apache.beam.sdk.io.TextIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/TextIO.html + - language: py + name: apache_beam.io.textio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/textio + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/textio + - transform: TFRecordIO + description: PTransforms for reading and writing TensorFlow TFRecord files. + implementations: + - language: java + name: org.apache.beam.sdk.io.TFRecordIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/TFRecordIO.html + - language: py + name: apache_beam.io.tfrecordio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html + - transform: XmlIO + description: Transforms for reading and writing XML files using JAXB mappers. + implementations: + - language: java + name: org.apache.beam.sdk.io.xml.XmlIO + url: https://beam.apache.org/releases/javadoc/2.3.0/org/apache/beam/sdk/io/xml/XmlIO.html + - transform: TikaIO + description: Transforms for parsing arbitrary files using [Apache Tika](https://tika.apache.org/). + implementations: + - language: java + name: org.apache.beam.sdk.io.tika.TikaIO + url: https://beam.apache.org/releases/javadoc/2.3.0/org/apache/beam/sdk/io/tika/TikaIO.html + - transform: ParquetIO + description: IO for reading from and writing to Parquet files. + implementations: + - language: java + name: org.apache.beam.sdk.io.parquet.ParquetIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/parquet/ParquetIO.html + - language: py + name: apache_beam.io.parquetio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html + - transform: ThriftIO + description: PTransforms for reading and writing files containing Thrift encoded data. + implementations: + - language: java + name: org.apache.beam.sdk.io.thrift.ThriftIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/thrift/ThriftIO.html + - transform: VcfIO + description: A source for reading from [VCF files](https://samtools.github.io/hts-specs/VCFv4.2.pdf) (version 4.x). + implementations: + - language: py + name: apache_beam.io.vcfio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.vcfio.html + - transform: S3IO + description: A source for reading from and writing to Amazon S3. + implementations: + - language: py + name: apache_beam.io.aws.s3io + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.aws.s3io.html + - transform: GcsIO + description: A source for reading from and writing to Google Cloud Storage. + implementations: + - language: py + name: apache_beam.io.gcp.gcsio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.gcsio.html + - name: FileSystem + description: Beam provides a File system interface that defines APIs for writing file systems agnostic code. Several I/O connectors are implemented as a FileSystem implementation. + rows: + - transform: HadoopFileSystem + description: "`FileSystem` implementation for accessing Hadoop Distributed File System files." + implementations: + - language: java + name: org.apache.beam.sdk.io.hdfs.HadoopFileSystemRegistrar + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.html + - language: py + name: apache_beam.io.hadoopfilesystem + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.hadoopfilesystem.html + - transform: GcsFileSystem + description: "`FileSystem` implementation for Google Cloud Storage." + implementations: + - language: java + name: org.apache.beam.sdk.extensions.gcp.storage.GcsFileSystemRegistrar + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemRegistrar.html + - language: py + name: apache_beam.io.gcp.gcsfilesystem + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.gcsfilesystem.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/gcs + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/gcs + - transform: LocalFileSystem + description: "`FileSystem` implementation for accessing files on disk." + implementations: + - language: java + name: org.apache.beam.sdk.io.LocalFileSystemRegistrar + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/LocalFileSystemRegistrar.html + - language: py + name: apache_beam.io.localfilesystem + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.localfilesystem.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local + - transform: S3FileSystem + description: "`FileSystem` implementation for Amazon S3." + implementations: + - language: java + name: org.apache.beam.sdk.io.aws.s3.S3FileSystemRegistrar + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hdfs/package-summary.html + - transform: In-memory + description: "`FileSystem` implementation in memory; useful for testing." + implementations: + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/memfs + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/memfs + - name: Messaging + description: These I/O connectors typically involve working with unbounded sources that come from messaging sources. + rows: + - transform: KinesisIO + description: PTransforms for reading from and writing to [Kinesis](https://aws.amazon.com/kinesis/) streams. + implementations: + - language: java + name: org.apache.beam.sdk.io.kinesis.KinesisIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kinesis/KinesisIO.html + - transform: AmqpIO + description: AMQP 1.0 protocol using the Apache QPid Proton-J library + implementations: + - language: java + name: org.apache.beam.sdk.io.amqp.AmqpIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/amqp/AmqpIO.html + - transform: KafkaIO + description: Read and Write PTransforms for Cloud Pub/Sub streams. These transforms create and consume unbounded PCollections. + implementations: + - language: java + name: org.apache.beam.sdk.io.kafka.KafkaIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kafka/KafkaIO.html + - language: py + name: apache_beam.io.external.kafka + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.external.kafka.html + - transform: PubSubIO + description: Read and Write PTransforms for Google Cloud Pub/Sub streams. These transforms create and consume unbounded PCollections. + implementations: + - language: java + name: org.apache.beam.sdk.io.gcp.pubsub.PubsubIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.html + - language: py + name: apache_beam.io.gcp.pubsub + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.pubsub.html + - language: py + name: apache_beam.io.external.gcp.pubsub + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.external.gcp.pubsub.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio + - transform: JmsIO + description: An unbounded source for JMS destinations (queues or topics). + implementations: + - language: java + name: org.apache.beam.sdk.io.jms.JmsIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/jms/JmsIO.html + - transform: MqttIO + description: An unbounded source for MQTT broker. + implementations: + - language: java + name: org.apache.beam.sdk.io.mqtt.MqttIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/mqtt/MqttIO.html + - transform: RabbitMqIO + description: A IO to publish or consume messages with a RabbitMQ broker. + implementations: + - language: java + name: org.apache.beam.sdk.io.rabbitmq.RabbitMqIO + url: https://github.com/apache/beam/blob/master/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java + - transform: SqsIO + description: An unbounded source for Amazon Simple Queue Service (SQS). + implementations: + - language: java + name: org.apache.beam.sdk.io.aws.sqs.SqsIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/sqs/SqsIO.html + - language: java + name: org.apache.beam.sdk.io.aws2.sqs.SqsIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sqs/SqsIO.html + - transform: SnsIO + description: PTransforms for writing to SNS. + implementations: + - language: java + name: org.apache.beam.sdk.io.aws.sns.SnsIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/sns/SnsIO.html + - language: java + name: org.apache.beam.sdk.io.aws2.sns.SnsIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sns/SnsIO.html + - name: Database + description: These I/O connectors are used to connect to database systems. + rows: + - transform: CassandraIO + description: An IO to read from Apache Cassandra. + implementations: + - language: java + name: org.apache.beam.sdk.io.cassandra.CassandraIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/cassandra/CassandraIO.html + - transform: HadoopFormatIO + description: Allows for reading data from any source or writing data to any sink which implements Hadoop InputFormat or OutputFormat. + implementations: + - language: java + name: org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.html + - transform: HBaseIO + description: A bounded source and sink for HBase. + implementations: + - language: java + name: org.apache.beam.sdk.io.hbase.HBaseIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hbase/HBaseIO.html + - transform: HCatalogIO + description: HCatalog source supports reading of HCatRecord from a HCatalog managed source, for e.g. Hive. + implementations: + - language: java + name: org.apache.beam.sdk.io.hcatalog.HCatalogIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hcatalog/HCatalogIO.html + - transform: KuduIO + description: A bounded source and sink for (Kudu)[https://kudu.apache.org/]. + implementations: + - language: java + name: org.apache.beam.sdk.io.kudu + url: https://github.com/apache/beam/blob/master/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java + - transform: SolrIO + description: Transforms for reading and writing data from/to Solr. + implementations: + - language: java + name: org.apache.beam.sdk.io.solr.SolrIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/solr/SolrIO.html + - transform: ElasticsearchIO + description: Transforms for reading and writing data from/to Elasticsearch. + implementations: + - language: java + name: org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.html + - transform: BigQueryIO + description: Read from and write to Google Cloud BigQuery. + implementations: + - language: java + name: org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.html + - language: py + name: apache_beam.io.gcp.bigquery + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigquery.html + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/bigqueryio + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/bigqueryio + - transform: BigTableIO + description: Read from and write to [Google Cloud Bigtable](https://cloud.google.com/bigtable/). + implementations: + - language: java + name: org.apache.beam.sdk.io.gcp.bigtable.BigtableIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.html + - language: py + name: apache_beam.io.gcp.bigtableio module + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html + - transform: DatastoreIO + description: Read from and write to Google Cloud Datastore. + implementations: + - language: java + name: org.apache.beam.sdk.io.gcp.datastore.DatastoreIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.html + - language: py + name: apache_beam.io.gcp.datastore.v1new.datastoreio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.datastore.v1new.datastoreio.html + - transform: SpannerIO + description: Experimental Transforms for reading from and writing to Google Cloud Spanner. + implementations: + - language: java + name: org.apache.beam.sdk.io.gcp.spanner.SpannerIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html + - transform: JdbcIO + description: IO to read and write data on JDBC. + implementations: + - language: java + name: org.apache.beam.sdk.io.jdbc.JdbcIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/jdbc/JdbcIO.html + - transform: MongoDbIO + description: IO to read and write data on MongoDB. + implementations: + - language: java + name: org.apache.beam.sdk.io.mongodb.MongoDbIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/mongodb/MongoDbIO.html + - language: py + name: apache_beam.io.mongodbio + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.mongodbio.html + - transform: MongoDbGridFSIO + description: IO to read and write data on MongoDB GridFS. + implementations: + - language: java + name: org.apache.beam.sdk.io.mongodb.MongoDbGridFSIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.html + - transform: RedisIO + description: An IO to manipulate Redis key/value database. + implementations: + - language: java + name: org.apache.beam.sdk.io.redis.RedisIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/redis/RedisIO.html + - transform: DynamoDBIO + description: + implementations: + - language: java + name: org.apache.beam.sdk.io.aws.dynamodb.DynamoDBIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.html + - language: java + name: org.apache.beam.sdk.io.aws2.dynamodb.DynamoDBIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.html + - transform: ClickHouseIO + description: Transform for writing to [ClickHouse](https://clickhouse.tech/). + implementations: + - language: java + name: org.apache.beam.sdk.io.clickhouse.ClickHouseIO + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.html + - transform: DatabaseIO + description: Package databaseio provides transformations and utilities to interact with a generic database / SQL API. + implementations: + - language: go + name: github.com/apache/beam/sdks/go/pkg/beam/io/databaseio + url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/databaseio + - name: Miscellaneous + description: Miscellaneous I/O sources. + rows: + - transform: FlinkStreamingImpulseSource + description: A PTransform that provides an unbounded, streaming source of empty byte arrays. This can only be used with the Flink runner. + implementations: + - language: py + name: apache_beam.io.flink.flink_streaming_impulse_source + url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.flink.flink_streaming_impulse_source.html + - transform: GenerateSequence + description: Generates a bounded or unbounded stream of integers. + implementations: + - language: java + name: org.apache.beam.sdk.io.GenerateSequence + url: https://beam.apache.org/releases/javadoc/2.20.0/org/apache/beam/sdk/io/GenerateSequence.html + - language: py + name: apache_beam.io.external.generate_sequence.GenerateSequence + url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.external.generate_sequence.html \ No newline at end of file diff --git a/website/www/site/layouts/shortcodes/io-matrix.html b/website/www/site/layouts/shortcodes/io-matrix.html new file mode 100644 index 000000000000..3717361c913b --- /dev/null +++ b/website/www/site/layouts/shortcodes/io-matrix.html @@ -0,0 +1,48 @@ +{{/* + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. See accompanying LICENSE file. + */}} + +{{ $data := index $.Site.Data.io_matrix }} + +{{ define "language-row" }} + {{ if len (where .row.implementations "language" .language) }} + + {{ .row.transform | markdownify }} + {{ .row.description | markdownify }} + + {{ range where .row.implementations "language" .language }} +

    + {{ else }} + N/A + {{ end }} + + + {{ else }} + {{ end }} +{{ end }} + + +{{ range $data.categories }} +

    {{ .name }}

    +

    {{ .description }}

    + + + + + + + {{ range .rows }} + {{ block "language-row" (dict "row" . "language" "java") }}{{ end }} + {{ block "language-row" (dict "row" . "language" "py") }}{{ end }} + {{ block "language-row" (dict "row" . "language" "go") }}{{ end }} + {{ end }} +
    NameDescriptionImplementation(s)
    +{{ end }} \ No newline at end of file From 8fe35ade6cb48be0724ad5f055b4101a25e12f6b Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 22:38:14 -0400 Subject: [PATCH 062/151] add descriptions, links to guides --- website/www/site/data/io_matrix.yaml | 64 ++++++++++--------- .../section-menu/en/documentation.html | 12 ++++ .../site/layouts/shortcodes/io-matrix.html | 13 +++- 3 files changed, 56 insertions(+), 33 deletions(-) diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml index 4b64189ab730..804d93f45a7a 100644 --- a/website/www/site/data/io_matrix.yaml +++ b/website/www/site/data/io_matrix.yaml @@ -24,7 +24,7 @@ categories: name: apache_beam.io.FileIO url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html - transform: AvroIO - description: PTransforms for reading from and writing to Avro files. + description: PTransforms for reading from and writing to [Avro](https://avro.apache.org/) files. implementations: - language: java name: org.apache.beam.sdk.io.AvroIO @@ -48,7 +48,7 @@ categories: name: github.com/apache/beam/sdks/go/pkg/beam/io/textio url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/textio - transform: TFRecordIO - description: PTransforms for reading and writing TensorFlow TFRecord files. + description: PTransforms for reading and writing [TensorFlow TFRecord](https://www.tensorflow.org/tutorials/load_data/tfrecord) files. implementations: - language: java name: org.apache.beam.sdk.io.TFRecordIO @@ -57,7 +57,7 @@ categories: name: apache_beam.io.tfrecordio url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html - transform: XmlIO - description: Transforms for reading and writing XML files using JAXB mappers. + description: Transforms for reading and writing XML files using [JAXB](https://www.oracle.com/technical-resources/articles/javase/jaxb.html) mappers. implementations: - language: java name: org.apache.beam.sdk.io.xml.XmlIO @@ -69,7 +69,8 @@ categories: name: org.apache.beam.sdk.io.tika.TikaIO url: https://beam.apache.org/releases/javadoc/2.3.0/org/apache/beam/sdk/io/tika/TikaIO.html - transform: ParquetIO - description: IO for reading from and writing to Parquet files. + description: IO for reading from and writing to [Parquet](https://parquet.apache.org/) files. + docs: /documentation/io/built-in/parquet/ implementations: - language: java name: org.apache.beam.sdk.io.parquet.ParquetIO @@ -78,7 +79,7 @@ categories: name: apache_beam.io.parquetio url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html - transform: ThriftIO - description: PTransforms for reading and writing files containing Thrift encoded data. + description: PTransforms for reading and writing files containing [Thrift](https://thrift.apache.org/)-encoded data. implementations: - language: java name: org.apache.beam.sdk.io.thrift.ThriftIO @@ -90,13 +91,13 @@ categories: name: apache_beam.io.vcfio url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.vcfio.html - transform: S3IO - description: A source for reading from and writing to Amazon S3. + description: A source for reading from and writing to [Amazon S3](https://aws.amazon.com/s3/). implementations: - language: py name: apache_beam.io.aws.s3io url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.aws.s3io.html - transform: GcsIO - description: A source for reading from and writing to Google Cloud Storage. + description: A source for reading from and writing to [Google Cloud Storage](https://cloud.google.com/storage). implementations: - language: py name: apache_beam.io.gcp.gcsio @@ -105,7 +106,7 @@ categories: description: Beam provides a File system interface that defines APIs for writing file systems agnostic code. Several I/O connectors are implemented as a FileSystem implementation. rows: - transform: HadoopFileSystem - description: "`FileSystem` implementation for accessing Hadoop Distributed File System files." + description: "`FileSystem` implementation for accessing [Hadoop](https://hadoop.apache.org/) Distributed File System files." implementations: - language: java name: org.apache.beam.sdk.io.hdfs.HadoopFileSystemRegistrar @@ -114,7 +115,7 @@ categories: name: apache_beam.io.hadoopfilesystem url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.hadoopfilesystem.html - transform: GcsFileSystem - description: "`FileSystem` implementation for Google Cloud Storage." + description: "`FileSystem` implementation for [Google Cloud Storage](https://cloud.google.com/storage)." implementations: - language: java name: org.apache.beam.sdk.extensions.gcp.storage.GcsFileSystemRegistrar @@ -138,7 +139,7 @@ categories: name: github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/filesystem/local - transform: S3FileSystem - description: "`FileSystem` implementation for Amazon S3." + description: "`FileSystem` implementation for [Amazon S3](https://aws.amazon.com/s3/)." implementations: - language: java name: org.apache.beam.sdk.io.aws.s3.S3FileSystemRegistrar @@ -165,7 +166,7 @@ categories: name: org.apache.beam.sdk.io.amqp.AmqpIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/amqp/AmqpIO.html - transform: KafkaIO - description: Read and Write PTransforms for Cloud Pub/Sub streams. These transforms create and consume unbounded PCollections. + description: Read and Write PTransforms for [Apache Kafka](https://kafka.apache.org/). implementations: - language: java name: org.apache.beam.sdk.io.kafka.KafkaIO @@ -174,7 +175,7 @@ categories: name: apache_beam.io.external.kafka url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.external.kafka.html - transform: PubSubIO - description: Read and Write PTransforms for Google Cloud Pub/Sub streams. These transforms create and consume unbounded PCollections. + description: Read and Write PTransforms for [Google Cloud Pub/Sub](https://cloud.google.com/pubsub) streams. implementations: - language: java name: org.apache.beam.sdk.io.gcp.pubsub.PubsubIO @@ -189,13 +190,13 @@ categories: name: github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio url: https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/io/pubsubio - transform: JmsIO - description: An unbounded source for JMS destinations (queues or topics). + description: An unbounded source for [JMS](https://www.oracle.com/java/technologies/java-message-service.html) destinations (queues or topics). implementations: - language: java name: org.apache.beam.sdk.io.jms.JmsIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/jms/JmsIO.html - transform: MqttIO - description: An unbounded source for MQTT broker. + description: An unbounded source for [MQTT](https://mqtt.org/) broker. implementations: - language: java name: org.apache.beam.sdk.io.mqtt.MqttIO @@ -207,7 +208,7 @@ categories: name: org.apache.beam.sdk.io.rabbitmq.RabbitMqIO url: https://github.com/apache/beam/blob/master/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java - transform: SqsIO - description: An unbounded source for Amazon Simple Queue Service (SQS). + description: An unbounded source for [Amazon Simple Queue Service (SQS)](https://aws.amazon.com/sqs/). implementations: - language: java name: org.apache.beam.sdk.io.aws.sqs.SqsIO @@ -216,7 +217,7 @@ categories: name: org.apache.beam.sdk.io.aws2.sqs.SqsIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sqs/SqsIO.html - transform: SnsIO - description: PTransforms for writing to SNS. + description: PTransforms for writing to [Amazon Simple Notification Service (SNS)](https://aws.amazon.com/sns/). implementations: - language: java name: org.apache.beam.sdk.io.aws.sns.SnsIO @@ -228,25 +229,27 @@ categories: description: These I/O connectors are used to connect to database systems. rows: - transform: CassandraIO - description: An IO to read from Apache Cassandra. + description: An IO to read from [Apache Cassandra](https://cassandra.apache.org/). implementations: - language: java name: org.apache.beam.sdk.io.cassandra.CassandraIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/cassandra/CassandraIO.html - transform: HadoopFormatIO - description: Allows for reading data from any source or writing data to any sink which implements Hadoop InputFormat or OutputFormat. + description: Allows for reading data from any source or writing data to any sink which implements [Hadoop](https://hadoop.apache.org/) InputFormat or OutputFormat. + docs: /documentation/io/built-in/hadoop/ implementations: - language: java name: org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.html - transform: HBaseIO - description: A bounded source and sink for HBase. + description: A bounded source and sink for [HBase])https://hbase.apache.org/. implementations: - language: java name: org.apache.beam.sdk.io.hbase.HBaseIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hbase/HBaseIO.html - transform: HCatalogIO - description: HCatalog source supports reading of HCatRecord from a HCatalog managed source, for e.g. Hive. + description: HCatalog source supports reading of HCatRecord from a [HCatalog](https://cwiki.apache.org/confluence/display/Hive/HCatalog)-managed source, for example [Hive](https://hive.apache.org/). + docs: /documentation/io/built-in/hcatalog/ implementations: - language: java name: org.apache.beam.sdk.io.hcatalog.HCatalogIO @@ -258,19 +261,20 @@ categories: name: org.apache.beam.sdk.io.kudu url: https://github.com/apache/beam/blob/master/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java - transform: SolrIO - description: Transforms for reading and writing data from/to Solr. + description: Transforms for reading and writing data from/to [Solr](https://lucene.apache.org/solr/). implementations: - language: java name: org.apache.beam.sdk.io.solr.SolrIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/solr/SolrIO.html - transform: ElasticsearchIO - description: Transforms for reading and writing data from/to Elasticsearch. + description: Transforms for reading and writing data from/to [Elasticsearch](https://www.elastic.co/elasticsearch/). implementations: - language: java name: org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.html - transform: BigQueryIO - description: Read from and write to Google Cloud BigQuery. + description: Read from and write to [Google Cloud BigQuery](https://cloud.google.com/bigquery). + docs: /documentation/io/built-in/google-bigquery/ implementations: - language: java name: org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO @@ -291,7 +295,7 @@ categories: name: apache_beam.io.gcp.bigtableio module url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html - transform: DatastoreIO - description: Read from and write to Google Cloud Datastore. + description: Read from and write to [Google Cloud Datastore](https://cloud.google.com/datastore). implementations: - language: java name: org.apache.beam.sdk.io.gcp.datastore.DatastoreIO @@ -300,19 +304,19 @@ categories: name: apache_beam.io.gcp.datastore.v1new.datastoreio url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.datastore.v1new.datastoreio.html - transform: SpannerIO - description: Experimental Transforms for reading from and writing to Google Cloud Spanner. + description: Experimental Transforms for reading from and writing to [Google Cloud Spanner](https://cloud.google.com/spanner). implementations: - language: java name: org.apache.beam.sdk.io.gcp.spanner.SpannerIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html - transform: JdbcIO - description: IO to read and write data on JDBC. + description: IO to read and write data on [JDBC](https://docs.oracle.com/javase/tutorial/jdbc/basics/index.html). implementations: - language: java name: org.apache.beam.sdk.io.jdbc.JdbcIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/jdbc/JdbcIO.html - transform: MongoDbIO - description: IO to read and write data on MongoDB. + description: IO to read and write data on [MongoDB](https://www.mongodb.com/). implementations: - language: java name: org.apache.beam.sdk.io.mongodb.MongoDbIO @@ -321,19 +325,19 @@ categories: name: apache_beam.io.mongodbio url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.mongodbio.html - transform: MongoDbGridFSIO - description: IO to read and write data on MongoDB GridFS. + description: IO to read and write data on [MongoDB GridFS](https://docs.mongodb.com/manual/core/gridfs/). implementations: - language: java name: org.apache.beam.sdk.io.mongodb.MongoDbGridFSIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.html - transform: RedisIO - description: An IO to manipulate Redis key/value database. + description: An IO to manipulate a [Redis](https://redis.io/) key/value database. implementations: - language: java name: org.apache.beam.sdk.io.redis.RedisIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/redis/RedisIO.html - transform: DynamoDBIO - description: + description: Read from and write to [Amazon DynamoDB](https://aws.amazon.com/dynamodb/). implementations: - language: java name: org.apache.beam.sdk.io.aws.dynamodb.DynamoDBIO diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index a44e0455a7d1..93c16ce25b48 100644 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -66,6 +66,18 @@
  • Using I/O transforms
  • Built-in I/O connectors
  • +
  • + Built-in I/O connector guides + +
  • + +
  • Developing new I/O connectors diff --git a/website/www/site/layouts/shortcodes/io-matrix.html b/website/www/site/layouts/shortcodes/io-matrix.html index 3717361c913b..3fd1eef025ac 100644 --- a/website/www/site/layouts/shortcodes/io-matrix.html +++ b/website/www/site/layouts/shortcodes/io-matrix.html @@ -15,9 +15,12 @@ {{ define "language-row" }} {{ if len (where .row.implementations "language" .language) }} - {{ .row.transform | markdownify }} - {{ .row.description | markdownify }} + {{ .row.transform | markdownify }} + {{ if .row.docs }} (guide){{ else }}{{ end }} + + {{ .row.description | markdownify }} + {{ range where .row.implementations "language" .language }} {{ else }} @@ -37,7 +40,11 @@

    {{ .name }}

    Name Description - Implementation(s) + + Javadoc + pydoc + Godoc + {{ range .rows }} {{ block "language-row" (dict "row" . "language" "java") }}{{ end }} From ff1f9f1e482cc5a8335a069610fc5a322263abfd Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 22:41:49 -0400 Subject: [PATCH 063/151] fix typo --- website/www/site/data/io_matrix.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml index 804d93f45a7a..4368386ef59c 100644 --- a/website/www/site/data/io_matrix.yaml +++ b/website/www/site/data/io_matrix.yaml @@ -242,7 +242,7 @@ categories: name: org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.html - transform: HBaseIO - description: A bounded source and sink for [HBase])https://hbase.apache.org/. + description: A bounded source and sink for [HBase](https://hbase.apache.org/). implementations: - language: java name: org.apache.beam.sdk.io.hbase.HBaseIO From fa805b693464df0cf1e04ec3d7723bf677a40971 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 22:42:01 -0400 Subject: [PATCH 064/151] format --- website/www/site/data/io_matrix.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml index 4368386ef59c..f1e4c962f0e0 100644 --- a/website/www/site/data/io_matrix.yaml +++ b/website/www/site/data/io_matrix.yaml @@ -374,4 +374,4 @@ categories: url: https://beam.apache.org/releases/javadoc/2.20.0/org/apache/beam/sdk/io/GenerateSequence.html - language: py name: apache_beam.io.external.generate_sequence.GenerateSequence - url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.external.generate_sequence.html \ No newline at end of file + url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.external.generate_sequence.html From 37ac9ee8bbd4a9a90bf149a2d599ed735909e1a8 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Fri, 22 May 2020 22:43:31 -0400 Subject: [PATCH 065/151] update CHANGES.md --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 59082149617a..6445baba5df5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,6 +65,7 @@ * `--direct_num_workers=0` is supported for FnApi runner. It will set the number of threads/subprocesses to number of cores of the machine executing the pipeline ([BEAM-9443](https://issues.apache.org/jira/browse/BEAM-9443)). * Python SDK now has experimental support for SqlTransform ([BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)). * Add OnWindowExpiration method to Stateful DoFn ([BEAM-1589](https://issues.apache.org/jira/browse/BEAM-1589)). +* Add a more complete I/O support matrix in the documentation site ([BEAM-9916](https://issues.apache.org/jira/browse/BEAM-9916)). ## Breaking Changes From e3b6a43de2251c55ec142cbe8070eacb72d7effc Mon Sep 17 00:00:00 2001 From: Michal Walenia <32354134+mwalenia@users.noreply.github.com> Date: Mon, 25 May 2020 10:27:50 +0200 Subject: [PATCH 066/151] [BEAM-10050] Change labels checked in VideoIntelligenceIT (#11775) [BEAM-10050] Change labels checked in VideoIntelligenceIT to make the test more resistant to future ML model changes. --- .../apache/beam/sdk/extensions/ml/VideoIntelligenceIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/VideoIntelligenceIT.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/VideoIntelligenceIT.java index 642722562384..20e9145d5e78 100644 --- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/VideoIntelligenceIT.java +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/VideoIntelligenceIT.java @@ -58,7 +58,7 @@ private static class VerifyVideoAnnotationResult @Override public Void apply(Iterable> input) { List labelEvaluations = new ArrayList<>(); - input.forEach(findStringMatchesInVideoAnnotationResultList(labelEvaluations, "dinosaur")); + input.forEach(findStringMatchesInVideoAnnotationResultList(labelEvaluations, "bicycle")); assertEquals(Boolean.TRUE, labelEvaluations.contains(Boolean.TRUE)); return null; } @@ -73,9 +73,9 @@ private Consumer> findStringMatchesInVideoAnnotatio private boolean entityWithDescriptionFoundInSegmentLabels( String toMatch, VideoAnnotationResults result) { - return result.getSegmentLabelAnnotationsList().stream() + return result.getSegmentPresenceLabelAnnotationsList().stream() .anyMatch( - labelAnnotation -> labelAnnotation.getEntity().getDescription().equals(toMatch)); + labelAnnotation -> labelAnnotation.getEntity().getDescription().contains(toMatch)); } } } From 0f97499b4473e21742a2aa2875b512964fbad43c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 25 May 2020 03:48:05 -0700 Subject: [PATCH 067/151] [BEAM-9930] Update blog post for new Beam Summit Digital dates (#11772) --- .../content/en/blog/beam-summit-digital-2020.md | 10 +++++----- .../blog/beamsummit/beamsummit-digital-2020.png | Bin 0 -> 72388 bytes .../blog/beamsummit/beamsummit-digital.png | Bin 63238 -> 0 bytes 3 files changed, 5 insertions(+), 5 deletions(-) create mode 100644 website/www/site/static/images/blog/beamsummit/beamsummit-digital-2020.png delete mode 100644 website/www/site/static/images/blog/beamsummit/beamsummit-digital.png diff --git a/website/www/site/content/en/blog/beam-summit-digital-2020.md b/website/www/site/content/en/blog/beam-summit-digital-2020.md index ed0853c58d58..dfbe9fdd7c1c 100644 --- a/website/www/site/content/en/blog/beam-summit-digital-2020.md +++ b/website/www/site/content/en/blog/beam-summit-digital-2020.md @@ -1,6 +1,6 @@ --- layout: post -title: "Beam Digital Summit is Coming, and it's Coming Fast!" +title: "Beam Summit Digital Is Coming - Register Now!" date: 2020-05-08 00:00:01 -0800 aliases: - /blog/2020/05/08/beam-summit-digital-2020.html @@ -23,13 +23,13 @@ See the License for the specific language governing permissions and limitations under the License. --> -As some of you are already aware, the 2020 edition of the Beam Summit will be completely **digital and free**. And as you would expect from the Apache Beam community, it is coming in fast; the Beam Digital Summit will be from **June 15th to 19th**, one moth away! The conference will be spread across the course of one week with a couple of hours of program each day. +As some of you are already aware, the 2020 edition of the Beam Summit will be completely **digital and free**. Beam Summit Digital will take place from **August 24th to 28th**. The conference will be spread across the course of one week with a couple of hours of program each day. Beam Summit Digital 2020 + src="/images/blog/beamsummit/beamsummit-digital-2020.png" + alt="Beam Summit Digital 2020, August 24-28"> While we would have loved to see all of you in person, we have to accept that 2020 will not be the year for that. So, we are looking at this as an opportunity to have a bigger and more inclusive event, where people who would normally not be able to travel to the summit will now be able to join, learn and share with the rest of the community. @@ -43,7 +43,7 @@ So, what we want to say with this is: We will have a great event! And if you hav As all things Beam, this is a community effort. The door is open for participation: -1. Submit a proposal to talk. Please check out the **[Call for Papers](https://sessionize.com/beam-digital-summit-2020/)** and submit a talk. The deadline for submissions is _May 20th_! +1. Submit a proposal to talk. Please check out the **[Call for Papers](https://sessionize.com/beam-digital-summit-2020/)** and submit a talk. The deadline for submissions is _June 15th_! 2. Register to join as an attendee. Registration is now open at the **[registration page](https://crowdcast.io/e/beamsummit)**. Registration is free! 3. Consider sponsoring the event. If your company is interested in engaging with members of the community please check out our [sponsoring prospectus](https://drive.google.com/open?id=1EbijvZKpkWwWyMryLY9sJfyZzZk1k44v). 4. Help us get the word out. Please make sure to let your colleagues and friends in the data engineering field (and beyond!) know about the Beam Summit. diff --git a/website/www/site/static/images/blog/beamsummit/beamsummit-digital-2020.png b/website/www/site/static/images/blog/beamsummit/beamsummit-digital-2020.png new file mode 100644 index 0000000000000000000000000000000000000000..36025d3e44f2a8fa596f9ffa904043b01ffa40e1 GIT binary patch literal 72388 zcmeFYWmH^C6E+G22=4AWxVt9UFa+1&?(QzZ-3h@7?hb?&`fGl@z2=k%*9>prBA?K$0p@P%s^@*M|u3ufK!lKUkrl zU&FsybtxQy;#i5}1V`J6zEJ@UHMB7~~sA(otZMj2T zOW&c=q9$u_=QXwktgZ3dr^@g`s18hT&nD7_C-}MAU-A^ClzcgkGI9F9HGiU@zd>IS zZDjk5L<^atpciet4^}#eEBKf-`ZL(YAgJzER{cPSXTPBNrE~5UiR^~sLw`U{Z)K#q`zF@?U z4cr4!o;Ye348F=+AD18O<{M|at$nR?JU9B4Yu95LdS%yFGMSz$8gg!=`^3aUyT7H} zFI%3J3w%b{wz2aAlWeY}W+r03gHKpp&!{QJJeSn1TzOmGm0MWV6h-O3AHsXZyv-6Z z$+thE+Ot$qo;;A%fZyTARbZuJAk8$xTMU6}yZqyjV@wjDjY;)6+9 z6nwrpa#_6N$3T^@za<-@oj=uR!B;59#418qT91w&lm{m8eRF+7c`E_~x=P`9e{kuW%L(4=X zqiMKk5MDjeb6h3ftl33HsICe2JAY}Sd8dVPGNXaRs7D!|;Pg752My|U51ILt z!P|t{Z67Bd3?7c3Id7SstqV5;etf)Wb{^6mYp<8w z6kfHvmXB9?RG$@hRje>ng;cT2>gL4I!yAg-zuLh(J(JFt+1INa&6uqF!D!72+8?8i z<9f)>BpVae@G_=m934cI5`=x1i?dV>huCQo!=0~R$6sf1wxeU~uu5k=p>%h>yEu7b z-HvO7N+tOeI7DqvFgfye9ghDh1X91!Lz)QkMc$mdJGuS#$m!|-GIzMz?^3zX@~MeW zDEr&Sx$A=H``;B%7O?@)Uvquc-{>NrzhVWVBS^~y3JMwf&leghBa7e_N|3G8G+j02 z{yLV9gNLbJ?$J{xuKwhJq2Gc?aW+_C_L?K?Og;tMX3J4A^3Xzhs;Jr@edYP z8xbl^c_j)72WK-19#(Evb}CUM3JMBgXH#=Q6-nT~s=xjdp|W&!brfV{^YHLs_26Q4 zaJFFM5D*YxW9MY!xEwS9lo{D=RqwEyK-vUG5D zc=h!wl*`z=nmNBJ`&aFM8~Fdu@h@4zY=7MK7x(=$Hvgc$21gW0nC(A-7DYmrVoZjD zdJiQdDW>KLebV{%)0%qv-S5tq^>iT-uCfpsnh+||G9;v+h@VK+`T5mURBh34Zf?;5 z26Nxq&R8ps@z(U}ju}VVQU}Dfq@ashaSky+lVhfjtNQDcE@yHGl!Qs_h+{vqe(>F` z8;I=tOq^$(dqE=UuQJ+%1z_SnQEwvf|GQ|w2|QqC`GP-+=?F^#E#3X^0s(=4>n7|k zjIY($D=-~A;E%YF#g)Gajq9!ib|aowR_n86{Z(1qCv1I(H~(V*5j0&# zLz{P}2n2NP7_}M(O2(;7pTkzk0 z`G-bfHq3~3Vkj2g53!p>t+C29qSQR#aw<1bT(d8vJ0wDTGz2Jr^XSj#_fWkyu#UWi z!_IRIiI}<}i_=NQ>L3T6st@_ag5T6-#>0BO3IFP7M_6~PKrKnns(z4Tfz)?al{G=B ziXycN+GTX;>iy;L>&4{1xv={RDbQ1~oKqHQmEEeeOsiE}zSUY`+FPoLK4g)m{ia+^ zek`fpo8y0U;01b~Pfg@J>b`TBDeOV;6(3GbD+kQSQl7@9;+7e1E7=WJLT@uQ9A^vv zZq;X<%>BDkE-?!Ki!Gbh^}+%!As%KA8%8PPHb0llv(*G;K#odX&R`{F5IQk$hk~<( zJFU+d>0i3i5-RLAxyV*xHom*U<+jWg{kLGs9*u9g{S19Q!0IDk^gt}|mtni}h8qE$ zH4|jEjYltc)^Ep>1`$#tM6oP0`{z+d6Z0|yJMttTCWBbxb)^(UFOk3Z0pV{L5p1Bi z$s{b|%<&4kdgZH-z6?CWE+q|a6&mjJ0Xz0tco*{>B?Nh1*{IWjzxVYttU%8uSjyGB z1Gf>IT#2{GAWsvJXQUt=OR2V9#3yPDgNB7&3Nz-bffFCPfj;-{|A~mLGzD3t@rE^x zWARnDAeKH}hEieZxbf+BsX4fvLp*7##oWCkF9EQ1^W$&h=4Xc(1z8n5X~1R>SKC|< zh@b+_(;?q_W|eaRIR&)=UbaaaPo^f6*5(;a=q3K|0jZRxsCFf)7n&Vt$LTq7F2LPQ zV!-7DS|6uzHU3;Q{(0H1AiSO-6KLU=?G63cXgI=x#VP#t9MIp%kF=MhZ{`F5VUxoG z)so2w914WT%hRkR)=1_x_4$`BG{j|^8NUU%{S6q>b`d1^c(dVgSG72;_q8%;hE{&r zllX0)FWjv?K4DvMcgefuR@JQBb_|=uP9kuwMGUHyPA+21=nX`H{G@}e92y~oL?JD# zI{KcRzgPD90)+lH=Eitow=ylYKiudm zMzOxm$-tA1Dm7@2U(;kvJBefibT5{eO&fK>?j0!&*2pR?0)4RWe{KCGT-{eNf%hjn0V`OL+UtHvGkc#CFXbV9 z(0?J*5jJNLA?>?MDji*GLq|}j=U)L$F7gv`lLBHqx$ED|NIQJR+3Z7}g9nel)3IJr zQMGsH?cYOZE(oLGY2l6P{p;`aV#!yyP5F5*_E&320@GgscRgJ{I49umbcR1TPAlv` z^>4KQ$9OO-enoPR;>EA~8-J&Nc*WiSYx@87`v2z+v+4$W)b1Z?@V9Y9_NT9s!Fsr_ zLJ&@9B4dB9@yhC)#=jOSlG7{}A*3rW4X1wAS!Roh>pt!H4+?+nyY9%ZVW^ zN<7?UIf5zo2wsb2Owg<;OixvvP0%&)A*N0$KT6#sOH{p~p4f@}jNG26j2?@jPQA5C zk8i$-GK@ z5a(p?S6*yJ)7I;?T9fmmCR@kMz-e2Xt`ll&_`f5j7!le6Zx;54ZM*y$O*@!yIXmGN zNYQxlo&RzfCTkI7FXvX+&e39xMMhF1k!aFX>+nH1;fN8YRjRzgJIIH}$s4J6)L~`+ zFCs|-tt1T@Iaawv2OW5JFc5fK0fE^GXX576H|0r(aG*!m4mdG*5fL`He*55)RsX1DEt`6^l**%-#<4zcNb ziUln56(O+@3z2U9_y3i=r1J`JYdWCGmi;Mixi+D#uja+bHkp`mcExC05IWFp zOz;HP$$#;AlHnDkL`%wdT9XN*(=a%=g5N8RgBNs{wxfWVDNG)v9XSn$k@z?c+7&Roy)w6N6;Ae0g~F!e z`C$~r49oBAFMnhUSX}D<{s)d5-cn$wZhsv$S9I#2nGJj2Pt(ZDpK(RB5zJEKCz+xw zK2Vp!xU@mW^R4g;b5xl&!OY8xRg)=al*z49P#uteu}hl6sB|Dr$F3DUk+j7l0a|*8 zalz)p0I;#ePnWREf$W0`0P`a)JO_9*<{lpYYAb97B~W`PQMjR;P@w+)yzz%=P1IPVW%6yz6^wc#GLE=i-ZO zc19cwQ~tMiDqWc-rFuN$Hv^KqL8tyGIsrE12Z5d;@>xuc7%Pk2BkLBUCOg@@DC3`h zXmw9V#cMN)uN&{V7`AA-=ik=LfoZz_%*TORfv*#kmU665K1T8zjO^dA%XTOqPHNQW zQ{yT;_zNy(YV`$cidFH&t)n~JI-84KRf#$9jTwRL4u>KcsEkJT^QhU9*iZCU+&f)NNLG2k%o|Wb-wBN^ zpOLZ~W1-rWav(NXIF~4;013xV2J5kJ-yJqMJ(oS5u0}qETs7aVk+56U;Tb?6)dPj# zOC6*inKrlHL6R!(P+^8QA`#_7dP-=*QOD){qw+Cd zeVRJBu8br*5Z-t~chr@I)s`Vmzt7i~4w{efv&Bvvj)i{b+Z>p?gOPM6ejVe%DIiJ>88wX!uq+kz5K`5KS<;yn!yG19?@P@3BXYQi%H$awaGUuYVtXMX zhyM@}*MNyh%O9o?kYmDMp-o&a=6Wnv@TLyzU9Y*K19%HC${BNsl?iF01hndW=ifhX zG6iXoO)Djq0Ez2mygsWW(1Cn-e%REQeJB4jTliqSPCGsrtIJ+CKz15#H~0w4y8H!K zl>mZT3e&s8dpMa0hY#`>H;>~PQBfCUO)mOH873H}6u=B(c+up34lKyFEs(w(g@4&o zrAS3L_(4Cri@v;IWvA`DE{QjF#(G?Xn~dpwT5I=rpb1TG_D==Xvh6V3V0YMr@oGWD zEeb%GF{pMMWg9U9m=q||ld{7=T)tLvlF82&H!Kt6L)coUImYu{6;<8KPeqUH2%vVt z{kh*hm;-t;Hqjz^8$d~=kwfE3i{=$CQH_;i7QVZzPw3%nKr*%+M!3i0ltcQKF!O6( z*e59bH$9pDLGO56dF5l^hLosnFg_kp4NK2>vi?{sP8QoHGs`y#sj}btMUjzwK2GKp z?pab`(lG%z-1i!bN>+zx61u?C6v-4$4hudY%J9mP(n&PH%i=e*Lr+ zk}+3--!$N=qNr6R-}$&}LWuj}6l9f|5?;s+Z_u!4B5Cx4>(o(V+a{!SkcY#LM6R?I z=}<3gX67u&zsp+^4tTks`C*`(MTuk0G)iZpH>EP3Db`d3+b+5TcnycQT(0SV3&h`AZ-R2f8iDQY8z>(g@}-pc!Hxz} z8N!Ez*{~{Ey10Oo2kkYaasi-C{*nBwUXGy~mmce2=x82120I{!c9Na>i<{D{IZ_do ztBCQmLIZGOsCCGl%xZhN<0O|XM-?xlhC2SK%=ZX8d4JGnZR$}jVBOt!2mo>Ca!b!A znwAaB&eO5rd=to5-%j@x;^h7u@w%Bj5rh5t-aGc&j1R`9mX{6uG{SAh6cl_&p*F!! zmw6+%0gb~qZ_$o}{Ip`I^k%+Qu^yOw6G#a3XoX~26P3k=i$Q*v0t9-XU_5Q7f&pU8 zoG604OE}VthtzbH^M|sWkB7muevjbRCAjnO0GS5o)xHVH3XKVHo*JtOr7n>=A!{`a zlT&BO#C3ZXKmyv7D+l+P-Yd8xC+8;9aXBF_Li_*#Htt|>icDz5b98xk2)#pdTD2se zaY*Or`_w@o%SJ+x5*btSE?u9w+Ip9w2PZ?J3^g4`oyGg3IYwdb&*sx9;sb>lKHJN} z=0d|crjpZ=$v}5$_)32kLf%^iv$^S<1*Kt{Z4kQ_)u1(pIKg}BZz~J*;l&K;s5SF2 zNh&mL406N~$N3k^eG^AxqSzzm(~lI1Z0~)?@dQAgiD`--d>`ufvClzVU2mkEH3=m+ zV=vSl4^+7xDk@aS!mbB7*6P8#e-?7)Z(w+hy_{j)F`*l*`lsKtPjPd2AtzLDgTb9U z$k(6Uct{{f-aLuKtoqg;uEVdVH`I8z!SR9_SVSmqGEtZDy2e0A!DPX?r^Aa>);kSp z=>-A)rkYGTL)RHaTBtF0GvC%Xf~@*rt~fpp&$YOVnGiIy<$x*P>zWj*eV2d}nqxvI znnrvw#QM}&6X7~Zvb23;Wcgh!2FJ14Sjz2t;Fev&M+&*Vfi!RF!g&d&DmLE3z+Y)$ zLov0QADC8f)alT|;8;g_8W&#5sA&qA#eq(}Z$p!704eU^)H1 z7vRF6>+p&-BtWZy%u6`G=Gm|TJ~C})oma8oGa!M|A&q#cpHn5cn~-UZ8u1G5WI*0K zGqU(|cnZFBnA>F%M|M347d>S21|8R}uiTtAqa8siO(kboyiz|zS37xy_+Hr&;*tvF z;YDNZK>oeInH8spH5<0o(G}pKMI65J+3xJCLTuqbF(~K{2B{?bRFR;hF4+x!ApEW> zHlUW%!&umbZuqi%wJjBkT&?O;Xc99(9VNSZE4 zXw{(C>hT@uIpy*g4>!rl1s98UFq1s!x)sI25}!$v5(!g`if2ZF)?@hzOdVwO)GTTa zTXwNh43Ii1)nF{wWVI^HSfDP_SQCYm|2~a9z8#9z=Cj255u=@v-rZk8gr>Vhg{)*p zvOJ$M;A?E@s%BA5GWQfea@?|@Ok0c2qMAcAln>GheteMeTAwem{q3vjm`T&F(PzH) zV=+zn#|~kY-fI4i598w$kNU1G>4@pbrW$$NDhu1Hc=wu z7>%LAXv^ahl^~w*ucM?4eb=Ry`a?=%(-PN5hN{Q8cX!8w8#Hsvt?%FTxAAVuZ<9AN z=@eG!EIRFQOdhaFwhaoU=e;Xk;CL5w(sdf))^@LSxOq+gBB^3Ac0PrhZSr`x64|*X zNVjzRrS8fu#5Z&A*_I(@%lu|)ipDHaIG#gHmIiozp?c{(zul2qyS+n@+@Fl!bV%+n zdK?zsKDUrFM{4y>e1Vj&)$TS7@TXZ# zo+kUfHP>--_9zKsH}ksg4yj(5DqrEbi*<6bli%WTA3_1%-^v>T^BDw zlS0V}da6C~M}GK{uD8*=SH>cS{;t*%R?d6VPZiEaCHijr5%BQ!8C>DK)96N%h8D4; zd{mn4TeEwniwdAwV z{YRBdBE8{i5&M!!Fn>25d>ziVcS*h^CZ7ui>e0+jxa*d`fZNA2Dcl~M@mibQnx-dL zanCYEz{MVlB|J;cYu4Owk9t$VhU*(zZZ;OIOe$(datsvKSPj3*ilso=ygnxaO@V1XB^jzKJ}Hn&K6AN{ zftI^iY-_EySS{w2omPM{gyaNdAc;z%9eX`kK`^Sj#kufuS6bgK0gHPMUD9TIZI(`G z*J>?~c%!~?fQ{5iiNIHfe*#9(??fb&uPs>@t5{TU^>85iH72RH2^YG9`iE_9hsPSeTHs ziFB%jXV#F=w0R>(8)S-nh16Oc+%}*yNeda*JWTA|b`~h+3gVO*S{oTVYOJjIkC8~)R#)NDPFnTU za}WqnbMQGOz)tW{zqsD&H&09!vF?F2DYI z{_5{{c;fFie!AGLB(p}z{8lecfRV5@J>`VRcBSH7hT4F+GDe)diJKj(xM{LTpi+^I z07MZJQJ0cA<}*dEk^=qswLJsGB?(cBM5e;)lFkV#Mmj8b1A>q!;XNc5WV0K10&j1l zS8kW;{`rKry8heVwb>nNMUv5_NkwT{ikby)d6zt3fo$zS0+wQ15QGTNhgueJkC+tS2j{R8Z&0T z7yn)Vu+w%8o2;@T@O8AG=jX-Y1OZ6sM%#^*1%nkI8A0 z1pPFc&&&XmYkwPZ_2#g5wywPwR-W~NmN9^=onYpo+dNKlN z2_aFI^5r zU6Y`hu!o*e<$)X!cgZ_}meMCCn`WSdH%wR&-94YCRQ(} zY_i$`o|$OrL**VIL%TqPhOH1z24$H~DKb=ak$bcIQ~5lSMl>uTE=O`dxaqnuFVm%O zke4gtD7I)}@yr?Id3ORqDT!?WmrPfMkKw?$6y#_O3qs|?`yUxqI_h8i)_D2U zsedjot5iLXoQ7;N+Al;#w~cMP{rNUxN3ZQH>)5-_KETroH&-hl8&VwUVh6ue5GGFXE6{Qz-*O@>C)k#S zA$0_Gfm<00bl~@o%jYp!92gF9kyX7%1798?ek>Wur)cXJ>eTSFTtLyMdy!-kcFlOT zTEB6PCegQtovqRBd1w|e#@3Ey;dV*dXD8kE-3mF#TE@5?$$K%!P&8F~7PMxu+G$NQ(TMTNBmKD)SXNyK1dIlTH2pMYNuzWZ#Ao@3i58uVM zS*jhOUBW8PWE#=|-6Lj;E9%9+nz_@525SxEK5YxezYlu5?RBi#avp-EPa&etJjg69)T^?C14atf4>HY ztj5CO$%D+Ao2j}%OE9g-cbBkXCsF=+`StyDKZr|Oyk(=|XcGF-8|m1iDRbkaK&@Dl zu#4MS`-PQThdrOy6V6S6PoFZmh2Jt7xtkhQ1O|(xHGN*9-~${>Wh{oz@dXmOkn!d| znSDqD7@iXkAA)1YLC+b9@SF(7BZhr)B=f~pRJ3YMd)!H%Xuc$X?1QwJZOP`8U1l

    S0O7;f;SEGdXIzNG+@vP*0_Ae5SMB{ZY~=dymi1ht@W_LuG9T3g0G^FNavTS0n5!f0 z>h!^PORpRooiCC1Y0X^kY9)%-Zp=BC0L?hd*ehpYt|fPRIc%0Fk$ErXaLf;W9exx0-{yML!Z(T(jXcV-sm`V>hqW&CPBHSQvU`)_>G&UALa zr4;Hl7IiBLz#3`>bfzXjm~g7Q|GrYD|GgzX09D4D+(SEa!F-#~1vW?O&mm6q=H!y} z3P0=vdy=Wa4vMJ^&MqR2Qt_qKUWkYhQM`qYRIC_ZjwG&LI9+V$uE9Q?H3WU!_HE&o z5$Gh?uau;YV+=ECZ0-d3Wqd^xG@fvoO85+^D?1^zs09emQl5H|&Ielf+iEUW*#twm z{bbg#N`)D^|h4fRCOI@g8A!8w0-4SEPcH1x0! zKA~+H1oQWEsfN~yYjbfFB{}Veo3sU&s&1@f%=)`zG&z*8ps%5~Zb6$9MCw`1X*_>$ zD|hJ7@^FwVty(_Hgz!d|&c|a8CRA4EWEeLVA>9pg)YXN4sEmaT5*Ol-$$qH?bFJy% ztY}7e5@+4XiOW1y*T>GxMJ;`J>rD8)jDvf|R26wQ{dt$0XyEAL||^ z+=(JvD@Dm-F0c5Hz#*a;Ju;mgO;QWRd;WK6Cm4}mx6&-n$&2Yut3sY+kJ~bn@4;rh zHtJxHaoar;P z>&%i5?s(TW6AxnqGl!Js`L7d|Tf7$pca6t9+MDrWMff6J2G)TzvnOngMiWrE5a^*# z&+I_6iN<@3A@BNJd-0&HhG@*t3e8L@ipX#FT`z+gQyZ|f8$~UBLtq7p{0b>(Y=$yi z$9L>LP`foGbcm(lBUB{HaN=6#=w1C#3yxh~X%=^w zK`7_=Y{3xS0zK_5qSg$0%L0@7u^5E+Zt1(=m)4NGGK?oOX~R?s_T`F+q-{@*TQnGuWkPAcqhz!xd}M2D zMy(OL75^Opld+v)-X}LAlAWh}ZPa*s)8%C`)E%kh8r}vh9r1VPOb2|O{ikMI5wmr_ zeHW3(iqW(Oq2_x(MK%S`l#1>oln<7EL`zx|Od?kox|YW)yeHSV(ay1+6~d$RlaN~e z$WlIDI01x-;g-kPg(m}ku)X58Lm&_`>ral(=T2RxS6JMmX8p+;D#bsbLw}HS07{U2uhocB_H% z`J;(@eU2s$SCFaTkPymYWo?>AEh;Y=D9d#Hn?;h``eTaed~oqlZ}2jbEiIs$hss~g z(}2m6jK5Ion%1F*)l0@E7d~v7YLkG_LTWHg-k9MiE=fMv_DFR%r<~74G$eI5m!;o> z>1BX|si^a;)+oE(Qj>Ike2GIRbM!0eHX~Jvp=n)#&XfP^0e_kq!qm>6lr}1KbRJQb z?4YxD-DHA-2z`xN2w6Gaf=-GGNm4#Yy}ZId>8(pSgepmVumjUhg5_9}KR3 zAb9F@6n!(ZonpVyBGYV3U-}t$AqqE|@zV(IaE4dPM>+XX1pRIQv~z2b(ItPM@d&h2 zf(4l;^XtL!~dN7ETHE z>#g;Lt&aM^Wa$A3lECySS3vq#Q$v2QVp`_+7!erB$q2WvPxyzeQP!+bYN_sd!jm-e z${j136bMflEC#hobw1L1HPJ4Hw6rU8lYwbG*h8g+oM*GO{I+Z`N!#)(C2oyA*Z#mb z7lVdD*n&HAedNe?s=wYPcRxhG1I{hmotMr>FOO5xNfTp46oq;qWMoRM3S zU1y9y?Fa2Ez3fIdm8&Swxi7lm9Zw9oP=7z8;K!5x5;(3lDbTO>lGOm<3lnTFRj-T! zLnF=s#bNcD@`~LT%0yB>&#^+9KcdH>SkuS#TX9FS31Sh$iwNh{=E-4i#^6?Cz|A8J zr+i?ZQ%5gyeeU=@5BC&*gc@z~IQ-&u6W@d}i5>JFKtwafo=zcQRGS^z|2^uSna&0? zHO9N!JaG>@G|+%@)m~z3RLq>w&3IoU0w-{8d;%PRdh_su^n^7B1zNSWB%M)G3!#Tf zT=&u(;3=D+Z~WGwma{dnaa+4OKYo!chTiF-6P$VYxmp%Vw# zL{8bd?yTP+>Z%o|efYjN?wNF(Fpyw{(PGb%$bMquPyq;HIYPqQ2%+I(L7CBk{mnDm zA0n(VeIi`#g6DDc_-0V6TA*6Mj+BUPR^O5jRki4}^IM}J;4Sc%UD2R3ZG|}rG7EEl zq!*izvD}bbETaaEcZ3z~!DnNP6M_b)D-m{+u-c8xP1ldDuc-*n3ChioJAse1lQ$^K zU!l{ta=jwnaT6>qDUs)Fg5)*js}k>H()+&yF>T!tM*=)ubb}q2!v`7Li1#z!DhF?f zYR+}=Aa=TF$(OKg66E~K2Eol%BDuetNe>_7YD2d=drWlCT9=bjaFLo9_#$P?$H5=4 zPib85E?l@HG~2fMiPG=PpTvZ!suF@@Qfd=IQeW5A_Jr*hDFzZ+ml&g@JqDnIDl}r3wh`9=Dhy#f#s^nGCg#MikA~j z8(a(O45{+-mShWXs`z!4M)`7^3K&0>;gFFWK}6&ps%ew|aMFo3yD*%wmQ-MIJy2TR zy0x51$bqU2}v|GpQmcU#9SE@0podGG0MvFSM#=AKnCsWmt%|@Jy8%Gn5<~VH3 zHkt77oiwhlv!Wkf%#lcR0$WH?;QIYa!Si~4&si3?62F8^%cT6Hw zT3i@QY(3oLg+X%ZpC?iSJZd4!`zI$}ii>LQd!n;$al5o*3{|M7%n^zoKUs;Mg?MB@ z`}wb;Sdwu4o)Ybs z(vE=z7%(5wRk $RC(R&j=P_AKKaKhO?UUR8xWTyzmt0?H0%fAKfOEq|ng-t_}h5 zytr=>*`_2xTpE(zw$Vr zr_CmV6Q1DyAzM@{p7M=PnKmk)Z1SZ2b4J%UcLSd=JI#%t^z=d#oGOnFyxBb3w4ga) zPx4jTy_9@Amd$K%Mf=U}c{)y-`c*_&H&OUwGkN1dzln?YR=t9)`P&xSo9+%O?ej(F z6S;{1b(~%y6st3pEOn#`@Uy<7SqT4G-lde?p4HZD1ch?n^=9A{s8yh4OVEK;gRPX_sZY*{{h@OepmgAL(db8rhgUiK7OTV-@BkRfb zRuK{+&vX8BRM_SX|C}=j>Bh~lbn&sM;wn$d3M65mnAe4jn2Wusejb5N_PT%hcKlj1 z3%x)>07k<$nRycJoqRbq&(O(nAJ3LREMJ$?vc?_na-vx`0mS{dz-b`)w`iauJ^sZg zqQeF#brNIH>@q5H=0Z95QY+Rc?8)(66q=|UUUSbr!z33n&&dzE9L|R+iXP>~MVKu=YZ&Xakqj%k`uT3QL6@ycTSwKWUP{jX`Gmpq3~ z`*X$PkDbUjhwVv%##=<4uIsul7e=^#dlMw4?QrO)NC13ZjLQvpo#Vp$Ctn$(hqzBd zw7k-VQw%yH1b{iIv>;;E_SCV!k;1yc0hOi1;?kXa$?$LPQ-v)rg8N^y_KFdu)XfQN zDFm3`n`++X3x7n%ra}?{3$mLHV>SgN6Y*dRPL*A9d5sl%g*0wkkk)IUA355ajN70C zqQghAy~ql5F&ebHSQOhgs)DyFjMN&~+4) z7R6Q@UcG&kYi7z{Z0vh~y)0Dj)yG^z0+q+Jhw5iM=v>Ed2ua%v166cC8c9|RXj!3; z%cxo>D+qMym1HSR-|*qii8Rajz08S^80#*9Yuy zjj6Ydc_=}?Txl6rK|GNY2SlCzxiR+I8PjYZ)#uib@bHRJh<{=Tj%Tpl=nqNe8}XUQ zN;ed8>?v?ig|ibcw8s(8F}OV9ve`nhQGHL5EvK00v2xDimHyqtm}j=^#A0JML}o8E z3%kX=3U6C!P50P0h#G4;U7Zf%*-}je2(SL+cS`aaI%xtI#IoHE5`)3=#Z@ba{9&es z5c44bpAh)83Hz>2wRdsYN8Vq7EVnd^9TWVtbRKCU#?$SQ{8J}icPkso{a`ixS6*WH z@D<&;RjvjnU>mrdW}6A7{ybJHDMUyxVA4NkkFaIg&nwJ!oi4Xs^-dxpS%!8_vYmty z;FUg*NCXhe5=`fNY=ED@{xJo4ce|Ud$6Vc;W{%8=8k?%#=s_fI(~78s;YOPtRhB#z zm-=#{OStdwND@VMNqjVs)6hA0P#`B^5^rNKz5@Ja_`@~J7fpv*>WpwpFLG*f?q_R6 ziS>~w?F4^ZAA0GG$k;W?YW4W#|43z#8m~E?KY}X^vO|t!`o4pT$rW+4CWPC$eisMU{3l( zYSMuqW1hY>;M#wMr9}#_%;>RjeIQ!!IDY_zwO-%l;*n$akx9^0OXkUefKE<}E+h|W zEA-QH5RxnR)-F%|r7*|{%EUM^su2S#DsJ|B$~-=DKm>?z0h8}>e7MJK|9xHDXhf*n z*F8janu+tx(y|^FRM}<9mr>afK{tDa;Ugfdw{SQdl^QvEhzD25-C{5xR&r)gGV%f0 zAW>JGA~<*()(PBhml>m<5yB75JowSRY@{(}3%DK<^Ob36gy&m}w4k`Ddk*cHOA`t< z2qfsBCCQabHgf2xrVZn}B!_QNf=m64xlV|rDo@?b^JGE1#N<6%aim@6$n^&4KFqs; zvz~IT(?UM8%gqgA?I$qti?|BySa^iu*MJUd$&&0Wsgadf>W{gofpT)5__dL10s7x2 zBu0E*+*eiR!s}BK2mfkptxISa?u%R!pE2oIA4Qc0K{sr`(Ttp?fCy zWS4!?l}KA%B^`8?yRQ!~*ndh#ys>BEz*172@AqAsSt(#!9XWDgE6}eJMV$fK3D3@m zHrF)snq^vP0O-0B2&vsaJnRThmRBq&lJX%+tBvdVFKPUZ zl>%4!jGtAA2=m6-&tQAQ9^_%U+642iW|*@~h4}80&>OWcaL#iuC9o_MpLr@Pg%hk1 zJd+1l=x*-^-4=AxD89z*SYl%JhNT+_RcK~xO%C2jvi6!KE2ys_r~Qo8 z6wx9LNvi!_#xQbaJ7Y~S7RZi|glVT~Gs%gPIoId+NVz=5ADPg^KP423TynLU&pT*qQ^86C&sgssc?I7=7*T4 zu;NP(5%i03NvQ?1(;vn&HQn#g`HKx%rq}4^m~s7$Xh&8j!vC8R7o{-8kHoAY&SOQ;crj&oqM0>#x3;tsiawf?1R_v9n|ghi zVunk)n4^kpx(I8>)0;?$ANY+SH={)7Y49fachj-A=IJOT3XaF-N8fiP>Qf3uaDU@~ z;{aKc!}sk$h@YH?-EM|$^g`XruS2=yhq|6Ac%D(Z5NKjF9f~{xD%F!CE<(IMA>)VL zoDjI^xeU+HG$c;wgMRxa2URUL!M4&bT?h=`gceDLl!La@3BG25)h}-H&1)L! zii%t+3D2G1^+Fb`_fj12X z<~$DiABcUNI0nr5m-ZCt!mxiX&v1*eL8F%M1(1P`q-6zf_Gq8V;li6TSlATw zGP+rn&*x?L8xLo83La0!ogiGM{ODVohP3JX*azkE9~O{6wod-zYI7fJ-9^fy(MhH- zLu4xr1+&`GMp3NS&a14CV9iY2i=CEVZN@VAZnof8wO;*khR6{Xy>Sf zrIgvLNW1S71VXt9-Ptf!-+#ArDAQv6GY=@HblQd7h#m4!bZS zm+$r8_{CrO#mPqW%mt85nv&~aLtx8?^swGMn?lh!k)x&hP+#Pd83 zXnr^}T@GB|GUujAXDO7;^ZTu(>+UM)DscZys8ohOJ}U%^t(cdON4Q{{KbdCZ&gc^J zHs2}0H=zb6vwYU>>*){Llh#T+{#u;z>qoZ&Dt37}b`nJX_az3h9J3@0$u2D`XTp$D z>%f6zlT>jRr!lZz_$xg4+10qxuuPSC&n&D2^*h17nU;n5XSrZwK-vaA(dxf3Qgkz5sc@*wZ{KQ&?s0WN z97Kh^kWFR)fkpGveaeT!*kVahO4euScBhj$}u7BLPeNO^Q@L> z9JgKQ)l5GRzS6!O%q=$3l8qEj9ABi5$B(cJyQhpi(56VHiAB;?`X1{!+ z?%XzG$Zn}8p7%GPKy};RSdgXG<_G{e2$#I+)tR!ahAWvx0?;)gjvAY<9fW)(Ehfn|R4=9nFBcYAyZao8<4+4J7EQ0oJ$UgT zAT{{I`pUB$I);Qbcn&UXBH2dKrX5G&sg06mJ^G)lJ^tR)yN}Ux(^xjL)*Q;N1x|-XY^x%VQ^)H|Ou6}dF zqeyXEl;>Gv!<+QF*B_@hzx7=8Eb7o06M$=fdNcX{v4;9%+=~To))$fJHtCa}dcRJ5 z(GkHs=~utHTYvVxD>2{R&~ZALs&$0)X6;D2 zGJ){g%V;y`CvIommZionC@<_GYGw^##*Ql0gfgtiwI8aTN(RX<>Zt>p>be!bv|$Fg zcI)I&!Bmj)#!fQrel3KE&LkG;M>YpCHPyfAk9Gk2~^m}RL zf<2mb(UZ9eS(=r9Y`T$QLNPj;(|Lz%(jOl>NR*6GWsqn^>TP((Gi8XaVSFKE#uiD; z(?en2>?SRX#j}h**qBBj6-dvc5MU#O)yfd@Pi?G$D`F$g;e(k_yaCDs7G+BB0}g`OVIq) z7}UFU^)F+(`{CALgFBGt1S1g>v5a$=0d>#GYC~UC|8@P4Ua%OGfH;H~yd%`1agbQz zah-5@R%f0$rmL@R*FtRS$5GKn8I<;fBnBlhTrCuhwt;v!`2t8)Rg&v4ylw)cozFGfZMGNN+ z>&+{TC}ZyokG11LdJb;&WV>z~>d+N;*d&1&7?U-VM%vWf{%@&!zRu2M*wdIzvT748 z>$~}$Mp{`36SgtE7UFZp8OQ4W2Oo#R{DA~TS%~CXNkp023Xp>#KJ?t>8v=W%Rcia z{6kvw{&#(Ym$!+P1*;m%nWm5DQEXl@Y*y*4Mpo;6m%c_H_|U~TI&ReMxBp)IuUMf! z`}4PG&YYw!`^dkOhozV>FwrKeOlMmXmHkF7U%o4Ta&m zwI{;_XKieAn{ezN=*K4gO(*CRpZs$bO9kEXyE}N_7U|-5y;buUbm`p}Urt*!);_3D z{q5Uy-udT}m%FuL-6oxO`bqlq-}LB=SN;XWxL>WE9oP-?9BZ>%62Ga-wsaN}ig^%1 zhA&<55gm2Zkv!{GY-w|P@r#es{>zu@?H7GgcmJNjl_$RU{TJv%AAXD0ty`~q?tWM& zoOGBj|A+VMt>=GC_aRv}P2eBD?G^gJmtDZJ=0)9n%bi-hWTD>uo;RUM&FF)degU#; z6UIUP%fDWxm%sca`rS=8L$ITI;~$@;uAU~n?fj1*CC9`8;ni4ywqY=F#j};1@IG6} z&*bMqY7>yEVsP)wYYs1alf8iU!W-eWciw(ZlyUOYepXEk#XuAzazhnecimP;LONAL zeHN?th3mRf9XhlNc@U(@7)Jtw_zIn0aH_G+(E4p&ShIPIAxSg|VVO=lgNplfUb2?f zI3)a0ed}I2>Xv1!bC5<#R#d?{yeIb*6FHnEAK>4H%54yPufrA&VFOv#$(Vg4iW{g9 zvz^_9u8u@lMiW(LY-^C;61C7scU zBqYBczQ{y~x{so+Hch~`*K><$Ni^v&z=wj;W9~yFk@gkK^x){IzI+RkRR*gR`8P-J zolAD2;c4Bk#lXe3ltD6;#s;@wgls@>I+wV)3mHt{vc~C6l4av)HPNYiA8FH9zfB!7 zAXREyYqshp^(5?;N$!Y1rn1F_Lsj{A{~Xgbx`4 z5Z)oJXNA1hmbga7DXh&Zr#VGn(Lq`s#Uz2cFJoR|_1>LH>dU`x)6om-dfS49N>$c{ zGT3AmN`j`KPp2+y(mRhD(id-UR;H_2BUPl0m?%{-1<9f=&asTLsp8*vL_x>3px%*%U{&US0=+=Mpu#O z=Gu(%5M7&)rO@gXnYi_DGSh(s&rEA?s(WS8&!t_Vk3?ovX zjfAw{_q>6~Lt{GS*h|$n_y_it^tEsQqh9`slXd1f*X!#4x=Hi)J&*|~r13Ym=#8(t zlym-|3oiUCedMDb(EI-M6#ea|zXh@GMiasUD%R;vCLWhzG7v%MDLtZ(eB#YWc8}6$ z|K{KHnNR;C`|>*PwddbpAW_ zxqtX;z5he6)kPQlxf0pK^^QL|S0gxzA93(|IEQfrlcE3k?!W5nSDm42ulucj_LFP$ z_IIAIQ(tzxzVX%X>VxmQgmVVLE_FC&sGXu z>9h54f9^~~z`9F1K!`@(M0)9>x(GpKp;cj(CTI#mw#iO)^3zIcufDP2DxF!KxU5lB zJ_*!Q=7nMl-$LUBCIG)w$M9b*Mae3oegfTMn31;dhA!Dfv z1~2Rwa~pL6METG5Z`MUC9>w#NK^`-SsNLJ1Kg*L16gHtaYDrxy3AvPDsps)>5>-5% zB#u2LY&fe$98%~p%R9Xxv8ArdNN-CJjUsBrYK@q>*qz4MQiK*WlZhzHL)TGF)*=~* zhJ~1dP}D~VKLu->lTaUCyEQ~CS;&|T5P<+eltbljN30m?i$4g5LYWTfE@?@YK`#F| zA*EVy6v@zbn8M`kdox1_k|Odq9h;TY?o)+C)`&P#B!+Mi$zy>CAl1T<=GO*v7!eT+ zYaFS~cV*Z3lCkJeKt?WOw3MSWTgTC6T(5lDh)zAajC6M(3~UhwY@1xzyET&U*7vVZ zYSp^d;5ltl6JrR_1n{&QvT<|vc;z7~j6IHn*lP83(qd^7h0!|K&e4)(5NimL z*;EH%$~YQ5LKhxm$V&Tm46?t780`u6X1%WrSjaVH$fz81aoC0#hNwdWHI`=W~Is_|lI8J}*4R@Tuw4m%JpHE`0EP-{JRZY-j2pKKp%*V3&O6D^B5e zQN;pgO!O~72+p2oD@C8J2m5npFao@9!5)v9d>7J^cBbjU57YETyop=O5VkI*z5Y=; zoj3%nf>E7OPzyyA#S15n$}yT)Mu)U79cD`yoKx1x&?#0J_>5RYAiXh0{y3`Tcr6E^ z!gES75^Mfw z!9Ex1@8aYLagvMr@6&trrbE|jlsLV4LQ}Pvbl++@7z9#xmhtWNh7_T}g<~2g6wWvU zO3>U#MydZ{Hb{1q?>JY)km5%nLesIuChrr>XAqg9fryxN6e+jSTv{9w8|ywTG~1KM zA9n6!GF*wTz~rE%Ijs%Enl@ZtcQzrjdaz5zrnQ=|_0~9#`ciZuHht~->u)w{*#_`3 z+KUJik${~Ix@=LxEah5nllr|0mEv1L;?Xd=H z4YiS1CQs4auwK|Uq<0>PRG-0mV5prCh)FfY`EEvyI2XO`ppuqF)(2AtszfQC5epOY zjP4rE=!(0In3(4{wTJVY5sEnXikJvJJa8hv{F+efK*` zN51ed-F5dPy8N%duFd^JI`frAT+GyfcRkNUfwGjKqvyxjd_n|Nocjg14W^u}w%z^E&aR2kSq7 z`Vad5F8hGqe&L(-nae+_?_K?OI`Y^q9eo0(Q$P9~gza$D_xtL`-~Fq;^}RpW$^)1T z;|SeN-PbXR$P<6r2TccA>KS3*wK9GGSz4y<~Wnxj$um0ywem4j6 zj*2~V;<0w6y<2AXAp|}~RDX5-ZTjAKe#ZCt*y3ijkw^)M>>=*>30UV$o(?!*nTCgk zbnktShhvGL`D@p`_dKEuw!=ps^8($E{qR-a{e^ya5f{S)@1=DwjbsEDs$c^&6318^F#u`v$OZ;6UL_MI zZ~@vY!n3e7b60_r$m&e|OF8lu9)Y3!ng=5~b7`y2>Dgb|(keO{ zKt*MqFsXx0OiaQ!SxLK4R6Aw3y#6+YEOn@RtlI)6+e20E;J(8ckj5_DgmI*r) z%1pbq=>r+$H~+)n8w-oKbaurwu(?Y=xUQt54#l35SFdWSemu@C%hC8+z4$OxyXR*0 z)gKfX>?}Sg?~-|++Y1+i&|n)K;@?tsbS+#pnSZ|SFg^L4hC?Cy$YPX9PBb`+Vcx*~ z>SjZV9qt!z%Idi8sNOVpU%H3KRl;D|2%42FTVAiTx~sbVu#CQTC(8#TSu%!r8r^WiZ}qx2oTD?(Izm_fOlkfH9kh@7`V2EP zL})#X7||+O9W$x}4qOW9G9t%24Uu}o`6ub4AAcvd)tmM8ul*R(H>f2`7eIo~)Zc&p z&-LCv{TM{*aco!*(EbN4;aVeWBqp%Qpe|vZM#7^Ve9%%Ab0(=Zo6=5w@Gmdaf<@i> zk8l4_Ygb|8eas4-@`@Mhb#FdHpZdhtwC3>*8aa3+<3Ip_>R&Aan6;H$s=$i zV}f6?MP|^FA!rEX8uFYgA-F)@k!fz#<)8jJ_u?IE!hX4lIQbh_VOsD~9eD6kUG;rC zHlptN-N6iS(?-h+?--#Y?V3%S26;s*M68&v+rECY{^A4QWq*S6JG7R`k+qle-*^s_ zgE3w8UzYEElmLkIuDfF8yn z4)3M()X`-xJ=h;JlMyi6OMB7lT9{^5mu%*{b*9#VT7rn$$VRfks&S_MV)(LEm5>l7 z@Wk$=BQyK?!Trxr_S9pQZtJ2yF6r@&t2KVl&6>OFRvpxdO0#1=gPEzrO=S)_f@s-4 z>olzMwLvkED|-Z;O@er{7DzpnXw?B-IMfZ?g>nv0bi%*nc)u-7*%pE@*)($Hg#qsCYiO`wHiocD$*9m ztObc-7ITUOwrj}>gA=zmiEjjHZp3y=`xUPM0PLKxWky1{!d@`_eiGk1t%Lz}@!(3z)Z z^@`)?!Pz$gH|pJ3GtVE_{5hL-?#qnu*0g~TNL_e?Zyv<{J>-)5( zZ!XufLgVr=ox5aEO-KYS?z2h5>XgldU?pPi-mo#D&)vkN86s2VSyi*0f|Qx9XIP;v zwBpHLiruq0L3XXAbfUzBMV|V&sr|dKzdS*!9(h#X{f0>qUj!-ojm|me488vKr$Kmc zVIQVTw1HAFhkdwV2crpny?r;l(?K;~4HFO}P~E!dqH}4&!+P-xFAV3H-CXMo;-iblE39tP_tvM*sS6 zU(+48K13+6g-EdH>b2*e4aQ~Nd-uJV35*b1I;*3OIYMWg^>XZG@52$S3+;_faC!Hw z-?ZmgziIC%WHMi%N40QlCV9pF`mskJ*FlFY(+7}FKl;dH`q2Bn&Nasnf0s!-{dLVE zvrs;JZ|q)WTcMQkF^Oi$`k8Jd#0BzW3Yz@zJ^IvVF4GZ59<49@%UAW-1NUgj{>OlY zwFkR?l-|Lj#hB!mzL^O|jCZ$nE(>%JX0fr9yp0-`?EKIg`Q+w%DOs&}as+k>0khw2 zSj1M;fe~KDj4Um*QW!j*e5=W(Y@asJ)njAKL_73&U!QJldy%>?d52Cu=45p*T8yNy z@d}i92?tPlJ$T#iblrb{Q_CLtfsUNp4cWk^4<{6p233$$nj{gEQQmd?0~xmGoAj6z zK>{?C>(yxc8G8RGU#xHc^fuk`n~gYO%+nx~S&9I0-ui``{2nf!!GDOwiZ^r?$Gy8A z%j%tvr3p=DlY}w47-$>nwwMk2MhrsNKUqUhUu2`Y7M!a@EL`hfn;1ab2>CWQQ_!Ne zDy!usiOHMSVeRwO9qVUI?1g0D?!w?;u`esdQ7uaM>Y|l+I}@hn2lrX1DePiRx?}xd zFy!7&K8*Ad#;k=Us2}a0ugk6}D%oAvTtZkSkSfK=RSEKDdE5EYZ09*!?XuR;xK~)x zfyshNSyR}IHfh~3nu#3kjJ#Ko;)F0@)}VGN*%M5qK~#-2ZIwaiO|KczOOM7jl?WIn zjEcf^<0h@-)`R6v{qQ=7+fYUwT`Z~sQJ&1tw8^*4B5>3guuU`3MqJciuS)7$NWngI(7x)dJP3*9%wU>9EmwzhJgT2d56SBN z*PFu<8l1zcT9F>WSI@#$Vbe|CYD{<|NRuAPwd=}zY+!BI+^!C7LWSNIh??c!>eG%r zj}oNB;=$H24QlPFXrzCw-u0)i)|{@azH-IC1H@0*PSfo-e@XxIi#Qk2L0jw2ld-q@6~^O?|P1PW7ED6a|{a~2JH&NKo9#KGSV_$#m&u`T|NOjt~T8Iu2 z)A}_JV4`uJ-tyKz($9Z-gFbcH7qn>cAzHh-Pk;Fr=Mp93Mf&2uenTJo$mQI(S%)2d zh`xE{U+dBjy%WNF38o@d%s_Cu#1DhY0DjtXh;?kzUPv$w;3dY$g{-i)=CNT;kpAh5 zAEy6L=$#k+Bfo6+jBy$hY8yWrIGGcu?ML4N-WDdVn74NXfdwc1KlaW8K+dXM|L?S& zEwjC5Q+Cq}p@e2a?;s!`RYc(eig-b!tMFGrKveL8sEA-iKoJl^mEr}YMoK~o2?^{NMwA6Ks$=P-oEQ)LU*nR#SiBIlmDC!Ih6AZ3Da? zN2_k-d5V@K-6Dix48$b|E|c5oJHWw!)*{uP-ucA|^bq}lN&*Me*&BJg+oW@^vYqzW z&n;+cmM^wtE7!PCie;sRR$EhT`+o6QTRUwZd*LrrtmBa%*xpqboe*J4&iB(O>5HS| zPqHKBB}u2F3-Kkav4}t#IdcQ_a|RBuYY!V~C!T~D`uRUwYct_ham*bR*FZDgB@scT z%&~X0<^Dyv08V)^^6S>lVq4ItdWo9@QFLI)J8i&GWGmoOo$Q*GAnJR^vVZra#! zTSmC?B?VZruMyGjwWAJ-+py9G)P>s+SxU_Vu80<*aFx9{Gv98y3q2pWU|O(x*BQT@ za!F;NyliUHa9ILy+hcHp^0HNCzy1Diy?}*cJ7(H&UboUMf_~nU>?5*a8Gb?!$p!HC zAlA$#)T*Qe7|8&-6%Ye^`u?>^@7RPzs=ry>+U&}>?$iAx#sC&LB?~kF&rjzhZTG5_ z9gVa7bR1wI4m~voYmC+piSB6vyQn&*OX>KqL26>?49`}Ke0kJ?&d_-wf8QR$D-0-ry88uKq=<3?|9m;c~5 zY?Sj5=O4*;0pjQ~JN~3S0K9MDafLAsq4hS&i0(12(q=_s`ha5K3gifU!BH)*EM&vs zv&LqLz^7Ogth;hq%tqlhQXJ2<7@lpn{ZOl|#XoDK>;UL6)NEwjl#fq#G>Ax^x{tDS z;k=oHIgv`d75vp&zTaVc@3X7zd*D8H#BoR2+2^9m`{Uo+(~mFJgCxqSq5;t5y~NF1 zfoJ8*1*zv~yks%n@(G8#YUNzKHGIj=Jo~Hm^?|A z$6C#(VtZrOtM>4tPr`9O0;93*;BK$PMx-9T%q0SOEF z<+G_@{MI5n>WkCtoC{91mvE`}t#96Fc?Fe>p0w5D(ySZRh1_gCKh(K=>vFaYrK~@N z&=%26^cA672{yHkgmUz0rEP%s(`xxG_`MRS^l9}$54RL%Qmz&d2%h8cGFe1x24P-T zQ!SeE{EdjyUFIJLKpu z+0p-fp*;^Fe&%l%+5VLgr;8iF`aey6p3#%s3qdyk2jW7SuYetDlcmb0TJ5ff*|RUa zU_+VoCwzGiyXUUw;HZuB0I%crP&`@HdZq0D-on4w2qf`!+kqZ)VLnC|5PIj52jC!1 zoy3t&RO}XoO{=T=*yY7ztQ#a+TzaT*OrkM7F|-uftuy-Uu_g8PlfBCAs7m@8&?}D) z61C%$47|~1yO%WE>DveGk8{eLFVOaG`k>##gz{WJrQskYA?|H(FCwt@3kYSYe`)D> zbKV6Ia9oOaKf8Wjn{-Ux()IL0o6>H)JC3NmJqAA+{bWDV4H5oLO-{N%on#OrLy(id z)P-MMK!XE#R0+LkDN1mQwf2sKAjo~P|`API^{f@({_feQl zuc|BL#vb!}i@%J^4*Ir?4NX&f*#0=L$Hq+^W|M<(pYX?F1-l=JQ)>PRpSEQ5lm^7w?N97qt&@Kl>V;t}KO z`&s-$5L_ZqCJ>7U0Yt5Kz(Kp)ICO72@Q$#<4!c)k=myv&t z4XZ9h&spt=#Et1C^#NCIHV!%1Tu7xN$-L0ICiMWNdPI~df?R|nc#dI1t94kLLfN$x z7WG*Jdqi=efBC!f?Q?tV0vGwW_VDzVSObS(c&1HBfh}6N)Q&vzaGNxFs6F%aYp8pa zaUPvv)E~N|s0}^!rn_(PTdUZd)Y=cP{xRkLB5J9iPsO{hkc(N0{GxGIyp1tfFyiF{{P`cLF#+g`GDw1CEJw|rkB zDk{R=3xZvyBg?+F&@4FO@QO$cEOMD21P^Fvth2R~POxcv9_$*UyP?fqdVQun_xu7I zRx#Fk39q;c|E7O?=mqyV5fRu=uDsgzIsI(AAAQ3a_rd8MIs&c${ztJ!Pb2PfKpScy z{)4y{J`4urtlwW*WJiAezs&dA>-NCC_u1IdJ8~WUM8c?MqZ8UIxiO30)|irTyVL{i zcuEj!%!f64zwjo|^J;F^b%_7IEC zN(`$mdOB%2pRceZB0ZKBZ-Ssl2uOi$9^9Pa!8)5(odw`6Ado_~!F0l1l=9o~``y@b z25lG~!71n>sD=94)5p|vT5&&Oa;9dHksp9h?z&uih2U6FRs}{#1jRWO*IWzzMhecA z^^^7->E0lsWIrH4oc9(x;UL@+m9{xo$36vKa81D?7#d^G%_y+j{-pd>M{##~pUU;F zcj_LUmku33oInj)xRa=ixQ!0n(>w3RSmuzh9lQ51sF}CI;?yjFmVJRnaZnu-Zd?@W1vhRK8r*`k(-vS(!0e1S} z_T|~kSC-io*BohQo^`r?{l5>f+it$h-d=p6?XmY1JM^f}+2aqs2-kK3&9&UVb>V5~ z&{pC0Y97b=_T-~WY{{aPcEk~f*za$9+~&`Fjq`;UXyuK($EcI&UM;`#yTZ0mWIrxs5WFWVVk|B4-V;z4%v zFYm#y?RGo*xC?E!sXN(Yk3PXY^?cMnd3ODc7dyYzhwODJ$4mH3wB?IgSDb!xi9PSr6J6EV)yu<&Fp`*g*h={j`uc90~_apgHXqO{un7 zGoy%4i8_W(p9c`sR)c@LqHWQCv=I=8Pm%2;K9{wXG;qB|AEK|#iQ8?v9_p|E`&F5H z>0BADIkH8wc|b~z&mh_IngJKB1TO9{#H6d*3heH;SK5>*#a0_^;u@+bej>F=@0P`` zhH#V>)hVL&U^-I-M~O_FxN!r;kZ50JcfMS;pzz--31tG1#vRu#(~tW8Xp(C$jc*|- zjbpW&n$KmJsGbfW*VwKa6hM!D5@kPs&X}|^7y^ryO z`6G^6YlrF%coTOr({(PNL4Vu?#0vrCm650x-)1;+6+GY2kZ<=r7PYAp{8nAm1~(56 zC;@=6^1_GhE<2Ul*(U_-mfML=hNp(CoD|$q`rC23b<@UPI_Uj_fA-&=>6y%>OfLF% zV@dr>g}b0D@2b0%<8b6F`zrFq|}1=3D#ns0(8XL5M=c=>OoYQBypfE;XaD1 z))g3S4=ydWhh`O7Q6%MU_~glhU52QTlL&|wk+PsiPXAvSZRN?eKiu8Q>JaXt+HBn9 z$u@1D-Elp&1h6ZBQ%saB0Vnl{z)Yvz{EMgT=wpsXt>JJ0@11t*&G#b`A89xK@@ji{ ze|*X|w%Nq--?ZJQZI5xJIG=i!$pOIo^^Nzq01fv&aDzQK{VAdcuD0#r0*)Fr($*}m zW7F#=00lu*=H_F#b-!JI!+5*n&L7!h4?hFFZp7E^M8}=P?iHE={bIuB<`vl~r+m@A z`lZj?|32}8t*NWG(W5T0gAUopUWT)}{Oz^eS7r}Qf82K8btk*yuUFeM&-};!7w;1f zO<#cSYq67#6yd$6acw8}X4`GIeb4ry{Ey!Mg#G%aYY^45k>yI9$@SPH&)I|bF5oz5 zi<%j4Zk%`^UHZ9x7&uMMnqR2Y_=XVrx5ODrBmbloF%HJ(Zyknm>F#d(%UzeDPBaZq z7*E>`H+_S#2}cz*FT8Ez`QdfD*SpS-);rQbaV4rvYQ0?l^xk%w4$Q3>wZ zZsI*_(T_e16^@hajW_1nWf%Pw5?;t!hPM;ezZ_^v3tkpZJ8eJv-o|gaH$<~z!ZG({@ZT6;T$V2$z?qz{5iDYX*k1w`^)q8hufaB!@jVWoq6U7Hl}8{ zg)j{K+@8DHQ%}82FbaGc=L{vn9^=BTL1a6S)~5;s7e2Nh+W_y!*0i5g#sC_LGxRzq zQqm8WQnxsBRtvPQx)n`4PQ|PU|V6?sup|1XW4p-bxaFYNcZ*AFa-9o4g>%Xhqy!>YZ}j z(9NhXk0=i}o&-`@7(6JUcH;s&owRzKo!f)LXWsP!`t zg6Mpv;Trkn7n!k`M55yy4x;k#a$A}GznRh}#D6cM-X6l;O1C$&F>_J?H68DB8$s8} z{BfP6^ILHeJ#ZvHVIylx9N2yA!KbNksf`>{P3Ue7KsEw!*!%iw*c{c{^!uL%eAO}* zC)!IGn_cv+YwX(>oP|Hy19^@pYArFl`;G@~^tf7E#O78>zF!}H_&HlRf4=?fri<+8 zFCOA%K^MxrH(dW)qMGeu)Ara6U<=(x@AF@eueNWWca>dw>FMZ-?oVGNZS|^FyYu$@ zZRGf&wsgf3?wjm%bbnfSqkZ+1FA$Lo^#UYSG1La0erkqYe94{MlW%2}m3Ggab1QsylcL=U= z$kwcGu#qE2*rW*)=o+oJ)Dabr6)RP4(ZGnE`^{h3RaZ6GzI*S@5}vTT@jHCMx%Y65 z$fp>iwzaWVu+RLorx|?k<3H^(&?wu|X0-r|r~@1+`OE|e1lk!;003uMow+9Nj27p9 zwFAjhT@vNfN8R8585jwfd)$rd{YhIfxJoNrTS++wx3pNXU9ATCc9It(&k69-2?uNR%L2iEl2 z$^WZ5;;(VL?d+8QI)gaz^@!PK!08x*mNWPtSZOX*`s_dus_My0)Hz1;MhegnMd%|(bs+lacvPR1TMZiyXTfyV}-?Wi{remDep>R8fM zY~O!zh~2xm)T#=ztxmeBjMMi{gUhS?bvJ+dfp`Cj_s(Jyuevw7_wb;Qw@ZV|(SK^a zFXtwW&1lN$jVkyDpb>6qc2=+D5*VeWF=pqUYQW;10}tCJAr4k?1eSD&;mAKPF*JHO3_mM=oz zk@ty%OMZTIhoiCW?TX*z+I7D#f}>M}dQjT)Gt36pb*`7&yZ`EW0CowRgQm`0JAT|+ z`{mR*R+ZJvT$bNk0D%@rf3e-LXpCL>>@eJs5rTrg(?@7bu8YJ0&nJ&$tQUY1wVMu( z+L>ckSr}b#O9Ea2vd)bXhK@ z|9GIEg0yc!A{FW#$`jDzn(G6;p`$)@)13zmAw=%+!NDy;eBO#bU5WfhjjzGrDU1jo z@eJw)&X270UTH6Zpf7!8FQ>Zn_rG4r@qSh_vc{G#{WHHu*c-T39MAJO9)=HG z#`-FOr#_pw!)T|w`ucxXbF6ns6#}FbBG3p1($P*Ss3vUm*c$XmBerO93tUZ=q0Ayh zIY^1v3Nbah;ckM`ZN$h@q8H+Ow-rfbE*oU}(lfTBs8OOm;u5fou*CBHtMw+u{PVMp zC1{WOMs$YT85C&2aKW2z87U&POdB<|jnOt<(l}8Yb53)Ubx3}AQ@kg;AZ-<;Apo=H z8SmjyH{Q6vsX0mbWg;!2XW!B&u-Rs{V`}h4pKG&TUBPjT@Ww-}ZnePWQhVU>o2&pA zo(D|5h~uCY;eM7iGu z3-Uvp*WU-(c$6r?!x8mhvzUVsp#p(qXKSueXiocS1GUepI-+l4JXc7)+A#)~%1RC% zdU_P-BIS*VLdHnDz^=HvYMM_ui9NC{ZNV+zfGp18ed0FZ7~UCnxPiq4#Zo(^S*6LR zN$5m*jCx1Fx$eZB-0ZG%S=)d2J2P=NHbQk?#w_CQX_i4um<-ahb5H2;LCGD8W#vbq z4uhrMZge`6`4+>G^#c$5!zzcC+lc&f^WhMjOdan9QbBnD(r}%shYqp1^WU=BfZ|Ep zO|WXXy=$HY@H#P#CX#>^Z&CUA5;sR6TT#DiYRhdo()1oeO@<(9k$iE>#NEl%O{LRs zeOyWT-Wrnndr;a?#|*T=p!+uIr`KLyOX{~i5LtAqeW)*Z9LO#NoFcqfbEoU+6E;31 zMJ*FoH|#9)Sp?ik#^n2^_4>#?sOQw9>s@=_yf)zmuaZ14@Z)_S_WZn3CV^8~Vh32j zlLdH9h*)!DyJe%>eDq;NDlA5|fPEh8EMwV)bm5@XUYJ<~*o95R!YcP?X>BHRv~kq} z`$6iOxyNgnGyu_YJjvm%fB5aNJ>&M(Lkr=Q_A++?qgwDOW^R6q9e!A$-TYw8S<#o4 zP=+qeEpdpoWMuN?=VqkAEjTzQ&lnPb%D+xr>s1hpBia|X-_1$bj-{ox7b+NXJ_cGt z9YMip05(Vo@^MK&3=E9Jv8nCz|J8##rg6cQT^oIWdJV)0!2b2K4`BSDRi+wS%|fxPkyUhTcS>b!BYv!uWdx!TJlz-EpfY_Y>eP) zu#2EwVIJVPz`ENTZNj9ncFI?dC1Ar2_NPDI4bW<~gOB{0U3$rR7+;0$x3}EQ`+YDT zg0{A)hdj!0DVDJLvju21Lc+Xj7yyuUg9P3s&`6s5s;vd@2a6ZA^AR8p5+Zk)d!oZd z0s}TvkJ5??%G75omP&O2&TNFOmQ*gNtArz37L$O_E`q&uAzk#|$=5n5i|5kbbIzbv zbVko=i`74N0D>IC)y+JpL8kX;O;S6uLISjV+5E3iR6ti!cWKZcfH382A$)RiSw7tL zh%I|tVt(NCHuCm&t^{>xzzzscvQA^EY7a z++T?5Q93=sI$G40_!ci6Mw?(m(n*^^<39^vmx4L!rSENNOKbrVZpjIld>(OUE1H1f zpdV!c4R!{qlOu?N={_wcnidcdxWf(<4%C}x91!lGux_FZTzuz;yGKL_->1w7#aP#9 z4)D?BhP^91Igkc&1C;s#_#*VRu~S_I*vYoBlecsCl@VB?8>ij@L{xnuzHUMHD2T$N z2C=|aTD^9;`!1~=Ys-ot#u!3{vIt3>z_+iWbvc49sO-H}dP}{E+bF*{xG4(3_02+r zw~j`DucQd;D^)?aU=@~=h~eZjy_Sp zGbX(;$bFlXKAU})R)|r@vDQYEssVg+vH_Pm&lo@ZMBH!f2=_C&oc2O8fiAk-YWWGx zxvVi_fBUC^Z3KguJahqxRFs1>gRrgUfSl%_+V_LiT}>}7mbQ)*mf?O|wWh@GeWKU) z-yvbchIPZ`?ZOu-Vq5Z0qRYGE1pM8eSZ-JUIz~?xT5cAcU(I#BPMLksw1MIcJ~faP zKRoVkfA(xv8(|h!!ug+F7qLGq^xN=VBQ}ynKOrL~oZTyUN91X1o7tz{UNk(UCXbVu==Qsn9I%BV}+)+?^Q6OREQ!sPB{ zd+fV2lDHnk`smPh_1bmU{Suwp*WgSJC-4NFOkXSC2)ebQAo-HyrK}rYuz7{zyaK$~ z0O@_LfZaTRb*L=Qjh7TiGsd+ZIJ*vc@~rCDh8Kf^VuB!`7v4kJd;6bvnUjw+7Z@I> z8-yMBb?4Xn^UiZT;F9;PvSmeR1Jn`Z-csdr^P>N*!S_-s<~y*JvIOy8zM&+w4Cf}3 z^0cab@fTcFhF-m%jzg;fu44x_nPF~`&H&)`8i#w`A_BNl+n#`Hg?ImZp0LktN=r5t9_!g&C9K#0FHtR>cN+fAF|u67d9CMyRwH7LUipu1wY zE3`*wVE_O?07*naRNfz6>5{Bul|mRF^p)<;?(<~}5%@yN{~ou-)xy=q;U{sP-S#_S z;hY?N3gT)BT|jLFB$jb_VIz~-y=dE?O>6=29szXd4a3%O&rTVVV>RI()J^cx!1#3I z!TrIF8G4pYgeY|z9H*Nr&U-(%cN5?I*}7d7WPCY^gT{R~Qv|i6B=PKXql8)Qw&MIK z(al(tb5TQ*q>`3N0T$$5EgdMg=Vs>Gebd!8i2Qq5MBrlj!Y%IDanbe2<|IVb9k?kA z!kK#T@gh6+z?7X|?CKH^y#VSYU6iD#ERqHc^PK zVd1$=mxzb+?(zOb_`h(6Yi8O6dtm&y@<*tK|AU)a&5Qdk5Vw z(C&bH=GH;iGN(T6@6DiKpLUVAy?>wu0JVOAE`mtsb|-=F!?tFLLKwwP_epn3#6UN* z<0~#iYzc8v)RiRMnm9iEWPz<+pF~)o5(cm)m`>=~4R6nqnuH-l8PROGFmPH~T(S}} z91`cHKM%?_I8joH)r(OK209D>iIwOpgb>}w&i~dou=@S2vl>pap!%4sjuXh^+Of*1 zc~76X)tEQn`b8-SSPh|ouZ39)qo%9(Hjlrk-&*e6eTaCJN=wU8o-ZWA%u0^r6IkUT z0}OWFnb7N&Sx)-4MU@*L?JGu z1nMBynQqvcx%r9cLq&&GVVVsesN0S1vag+zWmo>9i6<*iRb;E|#KQ0OUcE&r^X(lC{yX@}Ku-NY zNBWiSpT-GORMOP)NF4vp5xaF}rH#Rg{vc>b7%p^LT<64Cdt_yq-S!G?NE^vpokY6Q zMb-x5y~;7TO#df;)8@Hw1_k_j(c9fz!EdFms#xey?kp8$ZZB znS0m&-qdG5=>2Xk=w)<_dxmeCuM?GxFnYQ9d9`d7;`mQyGYtpca|gQKy8D6OwSJ}6 zJN}^g=&?1aa5k02EVmMy!>Wtu| zI^m2|8k>UX;~@^kYErvHK|;c{5L%IkF*Z=(fH*|251`6&!S#)eUMaz2R6_XC&hCh< z>`q!MV7UTUK|Y}t>FSb$;~euVlf;2dCA-a!u~Kbbk3IX)?Y7g7yV^PDoNsRub>@j@ zU$B~SqZ}ZYpH@wFxpm3P%Cg0amtvKBsO>yuH4$%W0s~zXZfmW^JBqySz2oj4+ z6H;Lk!&25FjT<`&qlbL_Ew1K>Ftj4@169;R0|&Z&D@dx-dP$p*kENQKcWGRNEfS8| z>gG0^T+(hQjOwvMjMTIT^Z4%+fi8(b!B=1qvSw|NjR_)02c-ISz}GAGJBNe&ssH!j zQ}5jPG5va{p6~sFOe1S6)~`!SrL(fS-5eZNNyM{*`z(@6a1=>&wVDAKBy32ghuO2Q zgzX>qGf$ag-94NP!F`pwfsC)5QIO_`H|F2F9PhsBHn!|BT08xKl8`<5ue3db@74G0;`0}xfe247V zbZl1gO14>#ZXpl(0aj7Qf!f#)t9-+v`3+S@ZIkIiU1zJ?&qdc9_hsV_u8Y>kn7C@_ zXvTdld217rXHkTU^+-4vY=B)!{xSng1dGY|Y`4|lL62i9)1Grc#bbz%#hmh_v28~HAZ1c#0-oWSm+CI}x>(x1PdY$i0 z$#0zpKaqoNfcFz==8x$m;)cM<@Z$turlbw48g0*WgLeGU6<9Q{#yCtI5G`)KfZje^ zxds+lrmGp9rXWkAUM%!)#b%Da?S(x?CpEmUSB=MzH{Z}mUZ1P z?BPcqcX#IH<^X21n8<{a#lUUU=+Sn@8E3-b9YI*xB=uZs;l{TtcSs2cp%2a>*6n~~ zF{iyULcjsx+35MLUHrDobK-9b9Tp6+9>m)jJ8yB71P*Nv(?^U;Xrd`~Nnu`Pu-jxkK_u`-p<) zL+T}6XFp>sV~^|Qt-pF#rWfBi`|e*GDfqi@+yc)aIiq?>N%hp${AUL92WCfRQ=I9S zb!w+J(TBcr#Fm@feVlSOB3z@NTqt0MdXmPk!;;?(@DT4xH_}xAVhrIVWfa);VQB67 z)>gjC&ii%;{bvT=IWXo2-=hL(>|_d}GGxlt|6BQ}UYVqyA{qnl8uaWyK7(#{b?twC z;5l!+ipe5>*DfnJ-!-z`3kLFcj2OLg;JotIJ^ke#TwY!3PjXKfAy7kCdmOJ5lH9_o zDq-9s7}91z^U%E-x|^v%e?ho&aQ)n){pa*MQ$GFn>haF~gXv0!-eq#n)ESzqM_oSd z2Y)i}>#x(m@!)r5?)~(>w*lTyukAnln<4BzR?8?rq6pV2ZLP!XpO4P5sXGU3|GiND zPppON$N@b@2dEx&v9azz=cm0PM9glp#SJlAzA6R>wbd3b4%%A_!&unLhowHKC}4q^ zjY@&^t7J~2Z>J8yV3rXQ0mE%(Q=TpCXt8l9=Z3^_X1>TBlin%yUib#kICl5WxD8G+ z`V-Qhol;(54UhcT9t(8YPKO?4=U#Z3eQu{|_S)+&*us^o&;=uWEJh9!CT(Yj9eRjO zn7F;wH8(Tan{CZYPuLXZmp_qfz5Wg?iT#$BqH?KxC&u*qa6->KD+v_Zo|AU8p?iIf zFo#3!rB`NIM^gt@ueI!yDJX819C?x(wRI-%hwqwCJqNhkxroX0;BY0OiZS}axpJaY z($qH~4;=%9_5+sI(Jm1|@}PmcG#;c&CpofKO}8NR+Q~b&+n2{D?ZCV?R9%)6U5!gD z6mG)Rmk$SO80B!wY$da3F(x1hMwgZ_w^Qf3^d=QG9>TTfwzQ`A9qa(z?v-Nwe`L(Ato; zQPrJx^a1VY$l`stM{dAqPwG#1S!ZgfJ@i7*9(YXcnYLxi)uyVu$aJQS`RS65Hzz4i z<%j6GO?sDU``5EcdwkEZeQih$zDnogCm7Jk23m7DC%es#*r&?wd#KxH&%<|eDc0^T zJn7)JRv9(G$X9dE^&tRXeFBZrnkwm!?&238jXiN*skAtQein1bA>_BaX>i_vFsvML zt?!JHN(#f)(gCeD*)OhvVZnNE^)KL(=MuIZKhQl6 zNuhU7pZKusN}{GF4sHN-Gj*xj%DHlM^n)1!!z0By;f9sEPB*C|WpjZmF-IEoTPPbz zPJr>HU?(X*n?V4wd~*k(B?$pW=7VOq*9qLO#{{NjixOa*IRD|~&oiRhl5nQ+zwF#| zQaAaCm6sU953cpHQOpJ%>b-FO<*Qqsd)yF2)lWXkr9zW2u=VPtqEkkS&&c9_)dQ6} zjbrUhRUlL`qnpvD{vpvSrKKZOPJhTfQ8>pLJoh3dQ;5g9{V6v+)z6Q_|-clO_Tu zc7EGXdl9!LyA=ls7mF`mtz+nCGUJ^}uz*o!7o31_3J3(?*yNzQbzoV{9-n@NJ%t7O zg#8b-X$S3X<98a5dzMy95OpmdA2$`{RaQWxnfA5@Yr$XdvY9iiXvY2MAQm&xIvtqK z0XX=Wi~=VLKPn=k$XL0)Rkjyu5F_{9*H+gx+XF7lVVTo{kbW?f))5I2Xo@NFx#bRp z!6l~}Ij9jtA<|K-nlUp`r|=Q+gV?%ys-re1K}_8W2H|uoT-5lhM^j-b^d2r9Fh=9JWAP`RF(Z>DgKq~0{Fo6TU(D7R*e$ny@5m~0f0(4 zL~xQey;nNj0&ZGZGbC(t>q8drn>=9X>CS45$k{w&Jg1CyMhd}V?RiTPFwkNnprvoh zgra#iA(b8YvpPHTD*>A{iav?Aa1RR#i<1wp1O5=|NA)S2J*(Y@vw5r@O>4#E`p@ev zGV3qphZ&>l42=gCtt5+)(-}5fS!edb8&NxWzZ@GoqKvXN(H}rTHE*IBZfcvIdP2Tk zJP%JQzA{vWh<3KYu2LV;!LF!!SaTncL6J2M1T9QD%z$Dy;na77HVc`W>bb!QFT3uN zghwrm-y%E=$Ov8AkSGRf4!7$OHX)ummY!DFah>$a;0oc6wQ_3=X~UFxkD!yvtF~1i zX3l?@wkE63T2g)agft0VUH$gC$thb~Cu7J8`Z{FA5kIsUI?cfAiG>lP(`)p%T96&H& z(5dJ-U)EvPKe|_s2tuSKfnrJpi_Wyn~ z+phR-zz*7b5}a3d;N8vUN7pAMPH&o&Ou1Hg$}={PZ)QdhwO{z3eG5LXe#ihTJ?O<5saf`Tw!2iSyUC5ZCFD-tZ4_ygd6 zMSJUUV7Dtc&hB54XTN-fcE^aV7it`bfO;Wbp|~4JVwB+V(M9B&TH9_@Yg1MlVqv4- zdr@aez`c+P%g51oL7_GHCkZxcK{y^^79^Y;*zC5T-S?_|471qe;>2Cz3>Hm|L*4I< zEsb}%cEasaORw!bK5j?t$3l^X-(4|P#P-1}gA8}laEdDP-?kr|Gt$0uEryXd4 z>myVn@PhOxlC{bXoI1vK-#%&onG>_pGW2NC(ew(S?3KIPUtn4w-kVbbLoON`1r!68 z7xEu)$wt#J?hqyS0^(=^SG#3u!p{vQy@u4L9X9~WB{f3-`9VLp)PIukB@SX*oL!Yc zexSwu126~(C>M=Azsjm3`q=}@g43$g)|6-f=DgdCxiN|&4MwKothzCaaqyZ3^&Q8! z#w$Wd7=ppk*$o(iQ|h8K;%%fLkIe+bqqRl-t%T&I1ou{OIXk*})`3;NnD?k&13Gbc z)hei=_vBc1Nz#h5qe$xjW4M*`1Bh`ph8H2f6%g600|tiSK2Is9sF!=;X0m>?#b9!@ zX|tPv8I9{MmZA=#o6&)k4ZpGxf&gcQtgTV~t9o==FcPw&95~uADdkzgM=+8^6tI_z zPk3B+rkS^HLnZG(Y+k#&GzNgcIr4ObKW7=$VwaGe+lxWC>Yz^yPZU}Py$xPliDunR6;Vc);pZ$}&!fk?qMCE84gfDZw7pT1-viy1%| zF-%hWtQddc>dSPV}Les-4Gop1`*EU}S|Gwr}gt5xMha4sIVRUL#>giAfDgb7WK z0ooxM%C)(jOinfcql=1dkJuXf*+2he4=x{p^8awEsrJ(1npt2q8THvZ7B@){wZI^k zwyX9@*;mGPT6wUA8X{uC2XY@A4*6jYB?U6I#IFyjXo`0wjWY*N01pDN9??(3MfTC3 zkV-%_i=V8mA=1JlX*eEIH%KG*7f~12rYEb~9;q+3YhRMiINmxU*(N{bJ_xlBFzG@b zyN!tVKKD}cE?^;T?FiT*lLL0)uDv!TuhVky8|#Ot_aSvatNddlseZR95cB~!Ayy5t z2*xm`TmzV&THInkc(M%6M;T)a4j1h%u9RC)XafTrWYa!AKVns)Mt%9nppC1-<3V2o zxp9|Jsq&Wyl1)fH3wZv}m0VfPaRy9q=wai5CC4FvAI*E&xIa5D=ZY*(&ID<;c>pc_WRk|5zxNS}x?nKjiN}D8cwv@Di^X9*5 z;X-k9)=7cf^{wHL_?S#KR7-p*z@AIm1S6hKONC8s(&E5u9;#|0J=C58l$5uM^49sWU(kJp_!NfI z3wN9yIl~Lpgs9XH1d4eFAT-?$V~JoXP(?lpFcAjaQ&+?lt-Y0OAYy>;I7a1_HsiH4 zpXJQPcwF?e?l>4P}z!i}urMndU450SgHPOs zmi5|d-mhRX&0S&q6-ylvNfjTbuEA!qr@e)e>3KzGj1%<{Ma5o1F^6(cdyzLNU4xd< z)?@IA|2A&Zy1A#DdK43RHBP{oUJ(=NYJG;*R*kZ4X^U(D29t<6KtaPSC?8!6?mj{` zwLqN8Dk^Q^k^=kcnQz)P*S6S~jw!TC-1~_0>1SdGSzv*ZtVIxb2w6%|*&zBUcI9aX ztL8p}OLlp9U?>aXM=<~gIACK&U^QLeW%VrqTPfl~NWtu^gx$F$U=!E{7mvb~5}+lC zC{#c)joxJvr`A5ef_A)mS0aRtPYm}*eH@3n;Ue_f(@h2(dR^-{}edH+0>^j zQcP4gD$|Vk2_qdxw*)XWnFTh)b#fDXiyQ>&78(MRkI1UAEoKL8-)TpT@37kRLi!xS z+>b%B875Az%MX#%x+S``iOGrC^_<3v#vlYgDbXh9d~jX;v~MEh1zV65GoDpwP3dZT zVyVxrcnQC*ZKYOHn6MVaZvw(8#v(#MxMgHlg-1hq zf1W^gX0PI*mq8hP1V5E$rkym!Nb>z!2o?wrn`BQCdaC=?Y3`61PGW_AIL?U<}3hATuymv5VmxihC#zqZOZ?Flqrg7Eae z-c`SiK0}{Sjx_X54e1kWGNtlFDUaG-+j1r9%#S$7aW|$Zt=h+3+e-e>&mwu=mV-7& zjV1?ajlZ2R@xyxU{Qs`C(IfF50=KjW!_)g;Xt7`4m9&m_e7?fWkSB+>COG_gO?KW{ ze%o!Qd^`e)Lzu>IwBN40u?t|83-?pHv8bAmM`=Neo&OE9op!*m5Q$Kbki4%gh}uu? zO4*uaIaXa(WNYx+Fl9>AzIv3O@WTxlDf?{qNd*8= zn^@6?5qOb3@O;#M_4|4|`G^8L>e#TAmobOCL-y3a6L#$_oiJiR#z<7FD%i|x9@2+LuNEn3`dKfI;RdRHXvyI17e`0WYZ%-mSK!e`h2rNidDmTMJNIo43u zWxMRuXeS+4Yp>4q+28-sW}n|bKv>~i7*cf#nr1(_B-fg{v+TkD4cWp4>(T* zdjZ2~JMx%(t18a6g-eU==ePCQf;p&dFjoqSo9x1`#ckSlSyl`;ypxdMvzI094|g}) zoHq*bvz%|q9vC9b^i7u}eth$78{qx;+V+#l8Ig=%;uxm0l=^*$P~`Koxl7?1*}*l8 zw#&b_!WO;NVyB)Gv9Yx{gu>7(M#w<%M3Wc}$mLNAE{nH#BF5yr1HY^R35-2b0x`^= zY`C_5#30#0L{%m1(Al(hH!;{ztY6hrWPh633qa|$6KYDV3ZR=#H8SVoyhETD`YI7O z6gL1+Va0cxLOiCtYlF}@5&zmNe{*bjDH zC%Hvl1k+pkKADdDc;4dKNu+E-QPj%9P3+3i)C}ZF5Qd1He&r+kg|w)=fX*f+J&_Ps zj%V4&v#{~eNns>7{?ZMT57dAYiAUf8724dMT)SsUq5a}H^g|K16%kRX70yS9fD9=X zFF)L7ap5#ZGJT+)5Qi-+bWIvt*^Ts|+48#DQG?(FqD#LFFnC`8PK&I|#hv@;k}3tc zmNybA7y?-PSg2%ZBLk6KF^2+GX5Ml(LE+aB5|bJSjM00)<~sN%6tzVvIFP5Ti*58<;5ShlEAII zA00{8j+E07`wIYi3zWtOzHH%I>;5!c_L$pLdvEb*#k#K$XEY?#?vOjGNYphN7*1td z)smb?7Z4Z`^|P6Efk`|sFb;%5b9mvp8GfiEl~a8nl?~-f{`8RxL!K5F5aYW*HC=(v zXvg`YF95?y>Mwew&AMYEzh9$r=a!KGyBb%1q$}Ems?K?E0(+TPwMqNs&nqn(wS`4X z*@7U(pE5FJmwqe9hK;MY?_I%~k)u$(%!Tq+yXCr0+huITX1(dNrFA(rq9kgk9+zw5 zM)lb#=k*ZyqY}m+;h@K^w4eOIZ)3+~+blrt+%>sYU68=VWUfuxIcAstc$Lj~y^6AA zSyf4gopdO30X^M$D|2n>Lb%CgDf{A4eKu}PJsYkZ8(vamug^sn7L|%)4++}PAsu$? znVqQMO(KtcyWs1McKWG+#0IlP3v;mn>A`qAiI4Y)UGcMSd*S)GO`Q<56Ze5*`fRVg z%E-uKLz;yspP6YT`3XB@uWX_aChbPYjqO3-zQxWwF>PxJV(}Jbn!h+`H4|fY&exK* z=MKF#W>f_(6@4~uey8m~o1On(r~@E@`oYiD#t}C59d`Huy@W}Q*^=b|@P)i52YvZ{ zVm6|NH5+0!qPDTb2W|OVF@lrTN(s*SKFO8WZ6Dbb8ODiZZ-h~ zsDke9J~?SS?&P;2w=AY{OTLIo=_T4>cwG}~I(j?WTn7zFmJD)ba3pxkY7(2Gx zuKQ8kCXNo;n{(mtFGtdguJ4h11#GV!qV~h<=i8&t)WUSb*LtTo+nP@w<=iH_%}m<> z?`AgKr~jCUIfFUKR7H%KO3On5A_)MUI3^M$w@xXsg1J|1VK$DxFQRc!>Pr% zEYgWS54*{qE~#TmM(vo|LaQaqaIDV4ESd?qqahz*C2^%wh`Uqh0a0+)dv=y^#^Nme z0hIZ(#qn_ygh{Sdniq4G4`3P2ftKtPtU-cevRn4qM2dy+dp{qQ@uda_x>c;Se;S z%!giJF*;zabi2TwlFN!kSOzXq99RdmlCQ>`(jp=M zzLXSYSyPkG{yig~P`b*~8p$}9?!@}CnWR*bK2MB5F9%(PnpbtsqYC7B7H>=MCWQIM$+*h;8%oiDm`8 zT81Pj#b%JAGnBQVQ8l0~{f#3xdnvIQw~{3FGDW;}n|<{eE-4e#al&qJUH{o*=Ul94 zXPR&t2Wk7`by54$;l;LMMa+Kl2Q@<6e)8>38#_K||8q;T{p!!voD+oTwDV4_w{yQX z%zk)&98Rx9=Y96w3jn*Lv+UX%qxSo|HFn&9vlG66!4+J|@1NUaN1h>dhhdDFPUh7t zTisY;r(PDey7{`l#eRC(NS*IOzT$9UF^YX+aT5PX_(c>-B##H^>hGqtR?fO`Gu>ln=Bl(zj!QKvRLUdm5 z=dO4ZBe;~!c&^t@KUZU_oKK5Aa!0$JcubDn_sIDO>(62P!ofv0{pn6T z4&`9fhIG@n5~J(1U3G1?J@$eoPLEx1M!Q{kQNV=_zVzxkyZfOM&KKCZC%4!S&&{?| zPmbBe*Qp*AsGpUY9|Pb%w=S02Eq4U$;isGJw(F{`zAkCUf3w@lE3jvXp_&30y%lb6 z9=dJP&z8aRwcqvGo_m+sU+!tJ%YLf5s&4rs!(`XE80U|4mI)8X*NMJaAbbII zjDi@nz)5p>N8A#Kr`)_~`p73>7NX~TH2FJf zE02=}cUMIRcqLGoNaE5a5XYA;Ayl>Ir{wYpa4N7PuP6y_PKw10p!eg}N`N9EgBsrJ zV}T7~?HcL9kf5jC@_N_eAtr7MurU7X+h)_3Kuq!S&<9IOaLUa*eYvDI z7b|ub7Rgx{empp@)E-(^Z1w30fd2~W2(a;W(0?Ixqa&ypz@No3c+)=^xxf{PU#j6K&r!6LLGyW>musY3TLzg9YSj>vU`>l+N{1IcpGAS!@}){LyRRR{aVZ%fP2XJ za2qLYraqDnV@`T{0Qr=_AC}^VK5pD!Vyn|T+0$#X?d7=|mjp6l!#wyb&cQkz zdgJQK;XqQou^^{UW76@C1coOdsJGy@p-NUXTNGNB`+iJV%_OQ)+tIZDGOTaQsz>!I zsfTr>owcebz_@2Dt9zYF4I@w_fp5StYx;EEKOB`0sp#P1@&(ig zniKZqOWN?3*$$J&@Lg&5KOb|RBKF)-ZH5{jv@(#au-9K12G}jK5hGOoYP<6f#kOi$ z&=#;(3hd_R3s?@bU;VYzmZ0N2dZGX{8(&ssSWnO&=A$-lthin1Rxfy)IVvkW=Psof z*K+DJ%8zlM7S0F}8@tgb9)+sP0eggO_M1xqy8`!P*|3&i9F9BD6-D+xH}+a(uFsD9 zJan1&h0*s7u-?v^)eG2FAB;v7W4H~en#j9~>0h|ys9*f+UtNG*mATBShZI?1d8OU= zVzJ$NPmk52=zQUss(%-51ECp?3B%!MAZgTYOIu_Mh=?;Wacd$1V}MqdJh?E784GSd zkW=jQUc57upfA;e?(MCAth1+IWruoH+@@?_fGDRA(N5IjGEzWHEPc!bI&X?cFHtOB z5@Gym;Nr}kg+q)W5sE$lr^Ms(p@^X=06Jh5aGKj9Bkkt5>+SXWRy$(2-*!gfaRdNM zv@wfGn?(ljh7l7(5CoY4Er!b*&k4T(kmnB&xZu6WEr(icc`J32LVi%(3URb4fVhHb z`7ne77lVVi5b*n##qIX-WL2-m#;tNOv+>1F;-#+hq_&+A;~c+hW^U9yQ3YBV80cEHyT zFe3crR$iV1e+QM#y~ewJ;IccXqYa&R=e*Ko{_JQ0M4P46)bV2-&Z0rz0?$DJ;S;M;x1eRTKB1dyeb!IKhcd=2|WQKN>EvAI<2q zS9*GFS~c#OFpiqn9J0G!1r!6Q@-d8U@9p7EZJfoe_sZer`O*CqV1OW!VXtFmcVat` zi)*tdP_wCHqkZg1#PJb=)IcNuu`JhKc)Nt_1m(ey*NxYYt|*^sPTE+)Mx~uK&*(!$ z;tDRb0C9T5S^yMaNk+Xm=D=8rer_+pyG^}K-CbiOVZ(+i2nu5Gp*e7WBNonYb&@&Y zmBo9Dt!8LFYDxlxc{X7@d~%p2FTU&@BLuXTX1p-qbwNP$sh6|t$S?HR$QtEM5Ek^R zyMrax#^%vcucQE&!q(^FwE;I|34t{tZ@LkNw-oe`d!Ro-7$)BMEwx|W(PQ5|J8hSL zyVCyaq(0njHrkv;S@!#X09w}yn8E;s2BHba8sd!yAICn?CF&QHdZ5JiaCG%93>I{e_43@p`^}57@Tq>R2pCLI37=+MFk3fNQ%=4)E(e->?w!K~gtWv3Rd_1T*#Aj;M#z8wJ41@1GXP1mHO?ZXsf#CVa5q794K#I)hH<2H=MZeyA7;@FEr zppS^r=1zU}iJleVR$g?tBWEI*dJnF7;K(AmhSP^z9|*jRmDP{1ANY0$I`M&$>Vq0G zKBRdUE`}bPljS~IB-Sm1%Jgycqw1$}v%Z>&AI#A>K*XQK&7eYBGe&U54t+;u36Xya4=HKrOkb*tc7l^fz=}o7vTW|# z0<`GY1E@MNQ0|uMnAQ!{*t*#$<>qVLqujy<5jhOQs(*}e+LtH-i0T-6F1K2mXGj@B zFg5~C@7OcS))-1a4`lxl!U)^B2?Ibr{oGKSx1`BVJ*v~DjE>=}-EX@dA0!Y{lU;RV z%wA>9%SB|E-wT5W*PLntzobJVX(W`TsLL9Qqvj&8r}^na$2~y(z3GkYRQ4y(Jk9G+ zB9EGB%m4}b;6Bs~X*1EOxNFl=e(5-7J$(g$U4dqd z=x77E((~vlzKzM>3*D@`qN*lWX&Bqd{iml!mCYK6fwwr*%s;s`3p2k|hX9`>!6lpr z2Hqtf#=V_@W>;sg?V_j$_JstzWy>RDQ_edN4}Hv0-pzVj;2H_m5~MH*6CfrA826_M z_SB2LP>S_N^=jkUu23VqCykj}*N=XJaC}ynyc%8G(pJ#|8ocW#ofC1>BE@iIw07a- z-`|5SB^;-|M1}q9sV+WESYx)@lpP@w<@n~s;yepb+k>xF=c5v`AfcnUK?0c>aO;>u z{jbdYmae0?H9hD67KRZ!#VhQIm0dOyE7i%x-566PZ5Y711nL=qgB<1~nwQPOr#)yM z#SN2(55g=RLT5EDzzV4CF9Kj!dV=^VLEu$<@fDr~( zvN3d-Wf7l7)arq={sM2MU+Qg9=k7j?!)W7RE^f4-Hc}5!m|MAxQeP!$U){T*QM0k8 zarg6plxC#eokg4C-qH1w&Lvn#<)B?|j@unWW&Jnkc+--5f_L_?(B`7HP*T7iC{%Iqr2pFvq|xV3b{hOca(2=6*IDEon2GUt@5 zH_!D7DP^!BWp0L&-1JhXM)SkRSad z-NH__8z9CRw2I>>-H%UjRaA-lm=xt~qSDZZF5aKbcK}XW4;-z84A;cPcFN-1yu~NA zFBFz362}*94!V^+cvNUcp?<_@L=;s3z`7DE83mAKPAy#~zzfahn(ot>t3w~ZyQj}y zB{0$1-w`0wrc>Zht*PX35e1%Ctswk6+}d5Y@3BW8?}9%Pw(4P06A_T?w0(C>*ob1p z^Z-gFF*m}@r+@?Fk}`;1v$>R11zIm~E5IlNZKVWtk&e7`HOKk(Mr}e<*08$JuK$h7 zruX#Nw@+)aOV0@40U`^)8^=QoU=*~MSBCl?Dib-mxt5EkiTZk^P#DDKaxDO@@#=!Q z9Pf7^3CV{_nQ?~~fc|8mdZ#F&s;?*T(D(tIOPb&oNu)!3djU?=M%vhE^UO?u6DuQ6 z0Z;*E9P={XxCWz1k~nd1y)s}UBXt}nX;MkJUk1mmtkJkT3(BFlW|txk!Z^aPz*^y? zx8gBL?^!@}wsbV#b{ebqbXr|Y4k`!aNzjNkT#xO)b0h8{v+RXe)!%Gt(E(1w9rj%E zQGeIoY*<`BVRW|`W>uinrqs4@|61#eS6j^pfV}3ft+Z+5n`~^A*&8!B1iZ`cAu~F3 z=F?8DFF)-_nB$p+vya&ylQ8f*lvOsa5CRswC2^d~Ge2i845BIX!v8mOj^93v+vm5RcTBM8*m7cMn`rtLhM}1uQ>P@ZA``qv8_c8VYcp9LN0SxTq&aYD( z4^>ZKcViUYjak}+@#^P(c}D1h-Q5MC3QHfI{zYq7n_-@p`iBFs)TKxNpg|#KtW=-6#Tn!yfFl=$J{EjQZRD5R zNpUuk^~Z5K3BYKAo&?c@45Nn82Zt;wt}bdn4#5m?**=c9e;Rqpte5czP^!SsZBJ|3 z?*3PYedp2~yYossEPP|BEm*-M*nnScZ2DT=hb{U2V4FDPsG{ZTS4sRG{O*ft* zP^p^VXSe-Tc?yU}?H7M;w{M?VK@bg<({GQy6tlBV4%->WnEm;04R~ZwnMAN!?b{ez z)r^YUtc5Bo(G-&I6Oty`O8 zt+*FXj5>#GHL`q_KE?lEKC86SWrg<(;(gt!q>-hnVzBmgn<698V(;?S;HnAUg;-gKOGKfFRe57NY zt3oHz7qOsi#X;BRd4px8{MrNsB(`OovEV6?i0o%j_+zL@d@WO2+KSDr`_PUGG!kF~ z5CFXN$(7P4+zg^83}7dwP9Un5N180Sh6={ujW0G-D)Uy6L{X)3qP`|j1R>?~QQi2{ zJ*75oLfpOrFn#f!K6{z)(ZRkB+kU&K)r^bS!_RcuPj5q7N7#A=Rk)fkwnGWBaQ+!s z{!eYO1uLR>z=_+glO+WJ++M5kqjwb8O}|@XSAQ>Q*I$!mC%@8atD5{)QAseP(RleF zctd0UIE&1Gbp!;YnprL)%RrE z=C>gNau9XqC9-?HrZKc=Gn}LJ8kZ)#lXX` z4I7!X;bqu2NacXwMP9<6e7N-gxAz_Zl3mrA|Ebi~Irnr=PD&$b6i^@tgM`R=cgcb^ zuMjFK#O%6TNIaF75rT_1{ud1hK zGzv4_6V!d~Om)3_;l6vnd*8d?J@*7F)-9zEix5&Ttwzu*N(nrY)XLaV&jf;mAmW9@ zqI+9{T*rcrtX41xf>hJP7^==q9Z)?`J|RqV?J_G*Xlz!t!c5~6>Jd1(F0v9UqAT}t zMOKqa4NFiHh}5a-F*4HxWl`~Ej5BV+43KN`19 z=%asx3Cj}H3g7dl1@??*43bwf-v9tW07*naRNIzq4_YhUQO;e9{Q-tGU-|bjyXrbk zG@z*B-}UVTBh??69@sb!c@gFNJcP5r z@>3ouY5}sN)yV?eMF#XdzsvG>3m=dHsg}95sKKiw5QdS_G4!qm?98sXok}#Cg%qh1 z3-Pv^gteiJuMmh$EPplN0wuxALNp_ej*&j~l(m^X6D*xRJXA;Eiwssg&R`uhS$~ zpXzzKuDU(_JM7m$rT0WMNeWj<`c{$7X|R9H1gQ_lMU;6S13DVV!vjg4A`7A4P^7@P9eO(@}2D;8MKl1|hI%x=DQ)IRqmLeX|C zw+$PU7)7#_j&JU^&IY>;zpi($$G!s>mt#a>tEtb};w3G%;bF5|ZqH*(h|c^t#GUur z%9HC^VlUcFH`gPq>9j4IWuTQM8W}Dz7dH}7tcjpRIs3=&jM@)>QD;pZ&6Y%0_18C} z!jd1c^`PM{*c>;RhELfm`mxH~a&bZq{aP$-)}D=OM~@JG#U8 z6IN*8EPemfRn4|_M~z*311lgzkdXFFshG9Zr-3Ivi?6+o{>W3*)HY#FtsS=Ro`S8v z7vWP1LX4`w?RQPs5E0{6b>pMByUjK{l(#$9585BRh5OP z@Tjd?nX*+&a|Dwj>RoTj{^=Xo6MP%u3pRN|I%kVHJL#09ZQhZv8*ij8jLSvF{~{RV ziCWjulSZ1Hv0vW06C=esT$^>G($Hxw^|f}xjT82xU+u8xT+&3Kj+9+_O&vyxHALyt z`&Dn8wOt~vfSpBE*lb}~Ago*fNbB7K1bX&~mI)>11yK!a0>B|J0{&@tpx zBzNOjUuwCkxnsmuF3HwBCXky*pj6Mbd@?L+ z)a50oEuhmm3&ri0~o&|6jh8j$Bl zibD=VoAROw81NnDIJ9+H`c`75dZ%l=Kj>xj_%8%eG<6tALKr}Gw)$Chf}!7!mqYb@ zP7A4eZndvg_gBB8GarH4)uZ-5TuN#!GF${b1^8bsT(kq*4~fjL%jsh$&+ z&($bc| z$ZfeF6XpZpJ4F7oyGs?}o}mrYAo{4_sz2X!s2FuXW1Qb}B!%j-Vum+@GLlsJ7;b66 z&H}?ELS<3OoIUMn<4C!2<8_Br0)z#td|fp*I537<837AgamPipHK{ywHBZ_amel+3 z%5vYmTEb`GXn+)!yFhaQ1L<+=@0H|m3_amd>s|nn+nTUFyV!;F(2zRXXgh)p`LM=t zZUPA?9#CR+tPn7k#CiW02ug$w5bcBu&01Dk@XAx5zm((&rYNZ`1NK@iNHl97$nEvP_>XCSW$7!r2*^-e>mO9XPtOQm91<0$2m@EO-~4G`K% zp0`{^y14_mj6=<8wAOluYVo9i&NPOE>o*YEyxh(Bok#Mj^D@?v8ne^SLFYM><~a4h z2(F~B6)w##1CU?2Ht(!TxkPV4DOqBl%kgM^#g9p~hP_c%Ae(Y(E8!DaO3 zYSD|W#ehjKa4iEzJ`fnzrS#7WKvvQ)PPXNON#Q=xFNA<5xvs=RI6B13NqR|*BerBw z2KQ!!kF0CLsFz4FyflaMO&H`5kb=0qC1Rpa;#Or$WPta^uq>EHhlrH5*Iu(^*q*0WdC?qn{7-k;hIj~i%8H|8r5JGmjMj*l@WwoS4oOQ1dgK#_X(D_+EbGm$QcAJ zEdK*R9oD-wGI{_b8JJ}e!%IT1fkVEV#Xuw#-cRBVt`_))B!Vz4(SHIJo+821j#W2x zOqa_5B9K7Z10A-&IamT0(U5zwAUW1Gp-<9|nxKiGhb4CHXIqoEuup~8i(9E$w&>TB z`>Tlncmx7p=^tWYVTFKu3qO8}ZioEC^e@ogONw z3rL|}l{}(7hxu1@T=?_shI>%|pg%zVV*UYJ-ACC{w;-WqQ<7$V@jIIQae^?Vj?uXq zezlMM^cBulCL#s;W=|-4IPj=Hk-oFuD{5!8V+A9XA>>y1%~zzqES$+#gqM9YPNDCS zO9VI@Dz6aYqfOl)diSm>^9OwKb{j|U;W(wDrd2wrXCL}~j>`@9l zLp6}rNrJ7Y&k5&RaLw|LwQTfmOEnUL9RZIf&eDf3j_cm6y5Z>swf=Bj4ey=Fu%B~i zlLR>BCHl=o7^fOI5!Yx1#=3e``0DD**4L{x({Wl2X;n6(_aSYAFJAm5(HQNq#@=7sR(TZ&Xb*^+Ry2+62(Y{gpr~ebT zOD;;;H-EaxKJj^Bt@;)0FW;50x4yN*Zo6Z^-txy7;njDsQk%hmTKCd<$3HpYJ^pQQ z)JmW(rioPnNo>^Nst>nrseS7F(Zi`d(LU|3JmldV&3v^_*NPx(rZ~y+(6BA-7{!++ zPWO{NmSgr`;HjNNH!Oqa1iiod1~`YXhOa(-$X>Om#?C=stDU8v%`A<4X+zf5@$g?l zh~k9|juX5@;<#n$;=x(XQq(eD0<9w1^2ewbEf1Aqs7rDg*NdyRY}@gT_~rY`_K8P` z51wjchTq0}0aXa3Q40Ra{n*0NKP7h7D?lhFRr<+JrS^eRY`@^9wFnPLKx#~5V<=~78ARirPv zuLk1K|7i0hR`X3PAtoSm@=rM|eU;D;bqX)#KvdaRA!t!QJ(3bjnisw7wR+La@}eeu zN;!OejwuqD$W)bR>v^aWIt?_C%zALFi+ue6IleS;L#D}uR0;Gn&2AF?hOB7mox!sO zsZTQK5^6w3hnZyv&=afv>WU2av>{h@XIkV9m`N>S*Y^#u=5Ct zT&?M&uZ?iP`W%ALR7;Qa>8Ir;=)xO-*`&FeGLjjYx;#YN}h72qOcdsLTcZ*jOOQ%0Lmz zYAD+XWfXS}lV9$M5ZdZ^f2vHuT{Cr~8wZaj^<0zXzT)GWPqso__5RYi!Y3p)Tez3X zID&p`g|E^d%S=F&893>D-YxQk(?}jU+JxYMQ6B2UTB#$u4$fD*jY8^2!LzGT{aDcu zc}I|sR`ANyPi@u=W+{wpbl(j552?+_gV#tXzd&l?UV>;;st9D2qzqc&sb>-}x-V~M zt(vgE{$E);@7zIri`UuEK!Y_CTDW~d*>1f*Yw!J73@;r^t*wKoni;*%MTk#~gJc-Ek!Sc7T-om-MtUQL`6Es1Mr zRj)5Gc<6yvDl*H?vCJWtDq13{gLu|Jd<)4sUS25k^U&=W8?cK~z4qSKV|LMEh%U=h z2?m#~%v8U8|F}JjKJat8T5LH3OBQRmk(x>CVHxJ^=6dUHAizbix<~q_7i^G7VHiM} z;>>onyevifB&rTKjyKtR*PGqc8@KM%a_gPIi8pV-2|^1i+%nErs1!zP_)HA`g9Blz zymX@K%*nseax7JY6ty+h+8{U+^Z z*ocbMScv`=y=P|awv5;!9+_}2^|uNR!eZ*9atJ%sS5hy1>o~RyExhBfbeEKGt|!$v z^e9LfLqn7V1w?P^SY@}q!aiL90h@wVD{F`7HsM@BuXA;cnwfk38)!U88d}G>Swjsa$-C7>0e*@;G2W=YZx=YiY1i90r z?th%{9``0V8U@gtm02}mgU5z<*^6GFu=_E!bi2J2`F7V_7E_--86W!*&-s?W_0fv)`bT_4?DgZ8Z|yn#?HgL1vFE&8%ji zUWk5FO!`s0eIg?>J(<;ybSHVo;@U+^FNre#9vT+fSGSez@(rkApgYz(u>yk(3@vzh zO@^pHc$c8IarBQ0VcEAj@Er}@Pnlzqxjkl~`VFZfiwCM1PP(Qqx8pI)K*ZyV`cS(X1 zYDSbdghCYAQ~~5UfdRS}HKFlge2%u&fSNC>kid3ZH@S69g{IR&rfC8oV8Ln z5{+bRXuuX@RCLNIOYGL$HepbMv21Hokf=t_`4QYMOnSJEKA`55j(QWz+@ph@b~m`>DXhH?QqUJR3492u7md<{QKbYKAGQ-mxqoi z9jc7ck^O(G=T`STnQQbuVdTdi2Z2ce*JXr=?iQ0@t9#oNUYkDkK6 z%l;wa?+4#m*Btw#QhJ($2&f}VDl1=pTOJuf4{Ok#aUME53m~F)*b-69Fe!&Jgs+oGdwsnlw&v_E`}8N@ zY0rA*g?8gj>+G*S_&M8fSHE?3w-TG4woakr$5Q)X|6aT3;wARUPrlDiJ!z%=<~P^d zyZ-Epco4~1OM9aY6P8onA)+?b%l8P&iU=o;>O_$2_V?SRFFe)W_klmKGtWE;-8QVd ze{iLJ@csV}X(%MB4Z{$YZnb1v2A>9&=5v!;6CJWYf8VR@t#5lhaq7El>((9i1O-ePk!C*xogBa zx|(tS1W9KlL=!}D-ez?C2Kxr>rLSCT@A+SEwNp=9g|SP?42m0~^CPYZLnO zqvM<}>D3HVD6f`TyXtJ;o~&K)tR?obkG&JWspr_$zrDsj@PU7}?T@e=iec1%B+k>M zQb%&@GL!damUKr3B)Lx73bQbY)mt!dR_5UX92czmeiJ=kP}5>!ZS0d=|d zCcO($lNezfXNXa-9=)N1>e*Cg$9Kcsk{w_CkJ8nxZ{GQ(C3rhZ84DANCu;tNLm@GHc%E6!DLmdNH--^GU7;sTT*ql z3tgmgQv*x2IeTz76+>6B20~N_K3kdB$qiCSGatk&8J)Oayk%OEP7Maj+!6ccH~-q+ z{N@)E?;LkE%vK^yrBlb$0)4Gjs}|bnXRNiY+jrT$ckhHu5nV~fhOnsI3~ z#$@OXZ+wxx{q3&_`g=>3EW(FlFA}vIt);C69XDJRq2DXt$& zz}3(a+rDF~t-IrPti8MO35x1PN<>n%sY(|PuSr5rKu58?StO8`G!vrBFaJ|}$qSzr zB!rzvNl!cDbfg*E?5;a+hO{lj^^l05h7EMxAdiw*zu+Zj*vCKq=XToa1?Vf{&vQ|i zEnT(}H$CfY+vYyX+lWM!)rlFBxllW#lXXm3c%<-%vd2E~S8uU*{1HSO33MBl=xfhd z9VUyvxZ+n~K&nHsnL{!z3(_vsEc$WNBuVZUzVuFe!NupG?}%^I_9k0%<{B)yd+qLZ z>#d#NW9TdfXv0ls_y^+LT#s~jw9nr8r?0llE_-p%C0x3EneE)U+itsggEcl2P6R#8 zKrRpK5mh;(_K>7jhEH7!aD_D7YZpHIBnb8YuvJT12}V=LGocR#w|qBaOr=i?&q+&j zt+jIh=7w5~dm0JBK50Mr-gOx2Fu*fW2vMYF>sgo~lzzD*aO@G#11XO*7=JTn#PAxC zn;Z{D>5W8YYiJllhgj-cbv7y$E`0XO^__fiOGq_)FOr$poHvB0xrzX^fyr0K(hP*@ zmXqz)-#yLV|K4`{;KwKIyT6#Q`!?cCJhK2p^UmPFyNE7tp22C7u)rt`FmQ_WKqTq! z!F3hZ@-bqYhsbTgC`_Rg6$TL%2&7m^LKR-Ns$kD(hl1pAr-H+NEdk?A(eCa~+mCl6 zU2LthPF#|;bFYT$ z_v`FsFMc*YY%%1+OT{oS-*ev%TX*M1ma?@RC28-ZJ?-2j_FBlLqVGr=D6bwtAkLl{ zCAf&lS+Sy*maf4#jdoJlP(??R(WBZ(1|jk^Amb*f)u>!`b*S7hA%m^)UzX$|w)9Df zDnk<4Q8}xGSLZ(Y6wc;y?9@o3_N-?uw$n~t#kMALV^&1BrxU&57rpdZ>}#-{+b07# zV(9?g*@TVDx`qfyFL}u`ZQ0@$CW<4hlt?$(@Mh9sYu7GfYuswl3sx0oN+5-bs^0(( z!~JTzHaq96vkgrFCI}eR?Ga=WYh>I+R zm1SZyqofxV6XGnBcjoDv~N})JPTy-tvbmh zZwNJz9guJ}nEGvj3C2v@ZANwSiyeXEjDXyVCdwIxQB)fA3PCk7h7C>*FHggSEgl%@ zvLVzRC!{(B@ht@}KRi4dd#B#j$GH-f+iRroG)EDT56L*4P!^)K$r|ejkz{)8mYW*+ zWbBMJV?-)w8Mstv>GJ}t# zGew#4@ zx13||`ja=>g%_@|eS(E$9aUUjBUt9v z-Of4tbi4TZXW2Esei*{G7_y0_Gu?Y+Ad9p9MRw6eYokeDu=o!JVsM~OgLzc9P}55N z*foVnj0bLctMFb1Y#H3M8PnqduT6Zs$$PVADGII4CM+>Pp*ime6K`p%Ja!Cop<<2 z1=PiQKOlXAsP_%#t+j*Y)}2Iz`kOD=b-z6WpQc)|%CVA^vbVkcwYF-xCL?Bd-1UHc z|2seDdaxwa`p9_3{_|INa-=m(;)BaRm+l6v{Lg$vfG1Y(XPz&4rzs=pEh7}edV%+& zI0Dh-5kJ$H+TOIl=ozb{2Xso*?PODai=}$XqH$6c$yIQQVoZtSMESG{m>9 zql0C-cH2PQ@eTK5q|-iN7oT5bNj6Tz77Sh*WHB#C+n9R=sgO{i1Is;_`m{PW4*3%- zTPSSABoez=9v$JuMO%!-R(iN48K3bmSRu{5OSUb;H4xwBEPtF@hYk?x2)Kyg!4TP% z6j|gs8ev1Z4{C@&9>T-Mtt-TcEzUxMRm|7|LOP}xm=Z{YpN7W)i;q}_hq8%>G0Q_3 z0$POg56XDX{gi&9Dn(exmNsi{k#AKJ6N9@p+;1QH>re9I4C`9dX#e)DAKC*C?Xb68 z_G;U>ajRYa`Co9X3keysbP?W}O5Ai`HEHEt83-x{MzP=pfQbryEBM_FCvjeQG4jH! zsGn#eRV_lCWrRc^9<;Y!qI_bIQu&sZiVJ_pK%c`+dO<7UU4+S0G2ELK1o;XMVNMf7 z6k33e?0FF0dsxbwu(~ElrY2>)@L%@Qi*dfcl5JKdAf1Xf7ZP<2@BVNk+BJKB1#0xH zNRwSD#=Q}(P=_279E?SubDn+X|NhAO_73AWyDL1)lJ1^)#tM7J`D=6qkWp5}0f#tO zbBr1*%KeDoRACNI0TN9ec*be_1fDvSVsun_#;B^KNOkFmQW%VNrQUSZgibRwiWD9- znOkq##zzEJ&w)HywAyRm^isZ~v}@Ns`}C*3%nz+J(4WE-hX{_S^^A@&U{~8SI^F*| z0`m?5O;Jlh$VAn{7>#~>m|*h0(JC;jSRsX1ksfwvr6$ihSFR&ZpyY)21Zs?x3x0=%lF(dsm-!}o2ngzGR3U`XOJI1(Jfh>G5C z3)?VYL02w`qz_umagiP!(kwEP(p1YTJfiYzE2_fu-_gUTi*ep9!z09Qqb#kgL~Y`k z-B?@WUP^;2TEr0Z=)9289RCwx#LEa>L`f;DY59`HcJU<_+cm$v+4k+iqrmE?+wHfY z+kMNYDyQJmtxbV;Knv1v0M^w^j@tMByY1iFf3xuJoZc}i{WM0<}YX5fF(rxzd;>QuDYH<&dSsJ1}fsMly-Gu!1lxSamm5qL@>p!bPh4*ERS zi()oj9H!a0lzuUg{ug&(e zD`IxTP3ZbQG-^BdjzNS(QR|R_VDu(^TS&9w&u5AMy$V-Hix?oJUZ6oNhTcOMD~J>$ z70YCB2{vwPFji}WSm-=O0MmFYErsylOdEW)Yx?JqkZOR|ha1TrsF^`6OTBtfq(8Zh zS*VQQkTOf#H-2%(s3TSSV2R_1etYdFQXTr=Nei4x>6yM)z}%Wmpnds>m9{_R;&J32T%}JWtzq z?7%`gy8nWnHhXc+c_^3Uw>wx0)lTPur~A z=E%MtC!e6+Alx74oFls_SJ6xZqj}n8!upx~mhj>-99+nSH&e(qaxs^+-8-|kuRmuy zciurPeEegkF+h?kgOms5ek__RLXtJYRGuA`uKF15AD-Cx7>S%P zULzU}N_VZ?ce$!TxiPf+5cdftAuEUeS7e9Y3!4`H}bNe!)U@0a$Dk?;>n?D1-2(0>u z`dp>g(SJ8?+>XvEOXI}2UewcKmp<TfJ3wM3>Puy+X4(;b z6a^6V!|EJ*>I~m-1RR06fq>rhp5`e6(uzeG=b#Yt&z=UnxoUy^ZB|-%gPaN`^!a7qdsf=r}vJ^=*j$qi< z2GLSj;WIlYZ3$7oqzD|Aj-`92@lGj7Yc+^RJ9KF^P2aBWoB9r{j5VY5#F^o67bny#bbexX)@|&}lbZd$;}XkNmyev3@fX49#x|mVjma zscRP7>tFvu{PX^qUGuxYv)_N)r5Ist$Mqb60GhxIDY2Sl#%VV4m~kN8F|Oq^-Z_k*SL>Jb$g7bKXh#zwKjRjXmc%=VIsYUaA=Ve1;(77udocTy2s1 z`iAZH+wMi>2R0az>aL`Gk7d$bkHtM7?;g{RP1`({DEPe`0Y~8IBM`kUkN$o94#yFJ zz$K&6tWpZS)z$R13^fveFyts&QTXtKaJ1PxK1Ok8G(ix8nnt3mG~xD(!912)W&{1J zZ0oiK_Q-uTwr3-f+U-TlAPLJ$!Yfq=ES_bIB!l-@9_9?;J-3snU)*R1WLkcp07LLcU{2d_{ieO#%~n z9vFxCw6w3ZAAI)~`~BblkbUVZKVzb@!v@h$jM`uE?9=w#i_XMw=L7bEzrKv%BfC*) zD28S4vRpFFG#xUFXlH1;1M&|oI<@D(U)9|Q?>*!VswW@DcQjd>E|!&hw(GCIiy#bA z2OtvQ%9TiYCBfZcuX)`Ws5z_%`77l7Z?9hmcDQpxLawAzRVifHG3$TSrw>H1v7yOs zy6GN#+wKi_>m(x5#g|;j_J}ohblY=&?>tLajIi!m|A^gj=fl>}C}S-pp1}a+Y~dN@ z`sDr|+@??Ns>3_$Slr|AO5|rd0*=73MZgK~vBmos-mNllK>{V*lps{Pneu(9Fse!- zrQriz6ln~EH*7~HUcn_8st1#(EKFdCHc=YKZCMrqj3HQ4r={zDudjf!F~CD^uVN= z=%XY0KH#s`jk%$WvnKL626Xa~F;1|y#-D`Tqmj3K_RgmypQ|~%XT%}@OP8NUG_whN z*B^bsUiW%*ct7%w_S2u;ivFg2xnkT#$jI)l2K%FTyvbhi-!8P_!Cg4w2RF@MW*IcO z)CvIt$hX?A;V$Z=^0pA#N;lO?LDbs&i=f}~OTc$Hq(3t)VfA_uNKLr*-z|-rz~x`O z!8UE)9`4i7h*bBv=doRjLDAWe>W+xI+;^R(Ofv zjtO!%p3PW0%l6to93`-=H5nm18x-x%*wj1zZyZXs~a{wiD`BL@_~>sR!5h z+86%eNA~Jhe$f8p-Ji5uZ+igm8;y9`nzWTmS_o3{EY6jK{({vrv6-=ySBv1 ztZ0n~-@C}%kst?MZrVrl76pGOAyv&5qGk4EN`^yG>ub8A9;ud%$QgfwgL7S{zF+Mv z`h7S_*XcuDVY{WN=Lw|+No#ZW+MRdZrzat#GtOLY&wt4!w)QN!_EIX_{daG)dvCna z>WByz$T)jdqUvge&YImMOoS270R6j~dek?%`_uMm8XEk2%YEU|N|pYj z-V=RPKebMj_LqkVUS;wej)ciq^(_B(1RR06fWZDIe=gK!-qk^`>;13nXgi7!l!7eO z*5?V?Id0uci2SrVZSCE>^b08IPGl@Mnz77qmdH@jb0t^-mgfrp*;tR)1|mb%Lx^k9 ziA~AH0zOI0lZ1TD5{`I+CD0ry3mT&!&v|0aGltr>;RYMb(4CO#$``JzpXG%i79Y11 zfhcMx2knf85j$r|!fxC-Xv18q0lFl01MVM#%qAe*LHmHLH7rdVdilj^(h+8`lH@oR z{QQZCGq9sCJV+SBen@vu8dqx6aWJm}rZJF%gfNhmWRNmn5?!LHzF?ymQgwBw?RA$u z&0g_eS6OkiXp3-7wtVGk_V?I#zWsCi;D;`^Et~t%L=hP>JNdMetfphJjSP*`7ifE6 zoGEf}SSDdniw+MfEM=r)+Ca3ju6F6X(pMpuoh>bder>UK>3L&t*xBC7q)nxw7+OJ) zBk4eU>1RO=rrP&sT0*@BHR+S-tAda zJ@cHYnytTQBS9!`vu1)%9H`r_d^N#UUbCH|GEaAdrnyD*yFxZ+`<5X;-M3mJk=}!A zrr*^`gSIQ@O1;aIBj5Xj=z5?BU#Qy9iQRTdEC1NF*>~02MjOPY_uxW@i!S{G-6)28mvP^i|jbnHh;LL?Ylqw+U-8P~D ztC5hpxGI~HAZak0;|Z`D()bc(G9ym{|M1y&+sj{mDdAHm?K7*sYM=ScH`%|^mamfT zW(Q<$hwU2}wH2!fFU_8GL%l8RsI{&AsIN%hQ#%FQR3or`7%5ze0bD|Pu}ZK5PXZGe zR4F1P@K5q>WuG%R^B$q4~a zRIq8JZ|Z`>+ECdd^_uAW%pTdY*S_=p zYq@?g<$#2Wn_SZx~JYfI$kB7Y|ds16OT$| z!dGa#${n<=`|=4dPs$O_S9xX?fXe;+%Mow{ju!$>c#jwIGrn&$;PG~zmXq`&89Mlc z@S{XRi}NYvix8lGmaUQEHYKg6r@@x5=&-Yqz~*upjBO@uU*D(=4aRH`b%U`XvLsN% z#CVENbwJS2sBL{~Fpw0kIv_|tA#JyeHQMu$ysk!-LL5`71Vs%V5EA0C;hQ&Zi;%!} zvb1{dc+AdO00lx7!AcBxL~;odTrJT;kaeeO?req>eEZ_pH;@0cbe_5qyds30g=pL} zShVGh1?#HM+WNtg-QM3t5%En%et9iVN>V)^5?I#~Ms*yq>Pctx*wdeJ4oleyK6Un| zfATu(8|byK|I1Cb^$~d*DA!-L==YKBNvntaLm!z){tZGfQ-r zNf|4W|F3_!)gIZj+g2^5@eAXGH9p{>08)yNXE_ z{e~!F_uu!RJ+$FIq6DtAG4;tnpjE7y7EXDnY$!~yDluJexJY z1XDWvL2YV=*+2f}2fjs0L5*hm71hE~Ce7*tp4zAD^r;?a(~~3M2t1_`m?tdYDcweo zDtwT#@-m)2*hMF44-!zsPa1P|hNxqL-|`YxBrNLaP|h-=y@B|)wI-~yvkvbIW^2xn z?k}N&Cld_lA}Bc|!9zIGFGvHzq#+QF z!xpurtiCML)D-xV++oQ`O;WUUA!+jZxucxJ`91E>P*0G{|2TtN@N`3RBaY2J-6AZ-x7*Kt{II?A-J5MG zgtx>*Q%{-b?RD4RZQCE&#wP?I3{(8RG?<%c8AWSBDGEMF48t`#}RM@<`@D_c%Om| z3Y`saZ0SYn#T~u8RgntO<*WE$U{s0}3dT4{2tWk(xrtki(c z==1IVyUKRm&wgo3PBUAbZbW}JhhR0$Pzy;$4IwjGusu1V9%W(#uYimPt22f`4D2Ln z($F(q$4@mnAW&7Jn;ul$-Qj&N}esA z7o`NxR5@H9J(FSQZ2Spe>W^OCd^3>@0%f-KeCMRP4napz->$ggw|3F9&f{5&R$zMf zjoKBzyqROnaKT)U2plg2^ptygiXkBK z8sk%f*k~|GhUHzoq!~DPUrTBdJ`BnXW&G8k$N?l+!NUP+0WnBxJk?>z+73&k8!R(X zvi^M&w(XGti0dx9_s(Iv_wEu)$vNA!anyEfAFzG<2CTPt6MjCgwXuP5>!^K|oqg7O z>_spBu)X26e`$aC=O3}5i(X>iyKca4FD$jY#~bXP;hfz)T*NcOq^%n%+ikg$Jy6Km zat56m8I&c_<3-n46ek72N}_gA9d4!%J=!iZAQy4&n}qPDvq{QL01U*Ezu7fzXLdB% zy{Kb+Wkbm}4YeDZ8w~m6TOiUJzIajg4fyb9?kB+NA1x7hqRXzj@>cu6Uw+oEyWv6F zSSTtdJMLw~aluw(+4| zw)qjMk(lk?Jz5Dvore6%|F$p=hi3MNJo0+3F@hBq>(*`HT_Oz49(r)A-F4T4Y&CN) zxyPHOdxqSre*$V@)k<~VyKC4UynkcBOWIUhw`}9Yveh>+G1mk(z>z#Kw= zvF7xT&Y}9u{i?ulHFz{NHQ)yqiOw+6Q4J^vY7&q->A}jlJb^Sei4WQ&QYTrPC(z+b zBJE8k`DiF>z$c*tBKt4|$WhP_4ietk7{yQUZpP$B(mO33ceFLx zU3Y9mZ}T?VaR|NKy!9gm-?e)P^0voryK{s6^H+Xk?|a|3Y~S8d>m(9a9}|?|uz8lN zi7Kv*V@%l9-DH`uoc;Vierr2+ZH4F*ZDerLcJEm{8r#Hi$dG{W`}-Qpjo7(6&-9YN$(E4x)SaJrD3bVz=FXpMCyw-=Pi1t-TXt z&Dw$u?jE+yn>VADG-22O?hgCH+6$G@_F`r>um zs5xL9bgngfo~0yhfqbsc?q2sG-ZuJd>-ItWmw)*lec*1ZZR`kYIC4`r>*8gVpK45^ zveCjUIL8F}p@;7Qlf3=zhC9(8{|T7z-UH8Zbp-I`f0T}}OI5D+FGs);IKdDgeAa9H zu|B~XX=Kk@Hb^Kzf|6EkU082w-rYz2F7^9ZwGz z5s3?~7X+%?rD% zqcv{Z_l;YjZzucIcNRiK8!%o3E7VPv4x2}WKXBFMYd=) z>J3azc5YNzC3#F+$AX2Bp}38rw;Q8eN9AQz!Ko36_So<^_Zy8@>!gbZgId^cs3!0Z6)}Jo(+Dg3(rzgR#FjZ!tbwi z@(C2IJ$%|xNvemib5ow%=K44j3F*?0RgSk=b9D?QQrR75pV=fBm|uBo=C0nB+61963V=w)6EI%=)^#*z5(e;kqB9MT0kv9hDNRJ zNhGzCkXqb+q4OxeyeXC{MR=3!tAPmDqBy7E6cNEiH&(wDu!2TQ*D3%26bea1K~!VO z8bS)Eik60C6pI;K(0#JK>CJy;t5141gA#7X2t1*(#2~pMsKM7Q1A78VtHzD{@84vf z{M-LyH~jlI?Lw^Wi%1u12{7@J#ohLz)|j;<2Du7r_p+i>rO*jRMFWCfQ>6UlQ;ye3 z$$>kNpBMvD;W*_evs}9koyi~W&fAwa#B3btZXTCveYht}ml-?>Wt>N{O>IM&0@+oD zVbq zuq0`2;llWyc=t*qoZ@Oiq%rHYHENi!gLBZMsz@7KdKn3DBCm-v#*MKWNpx&GyiW;? zR|%;F?_QU6AcN;6ddV$mC_i!t;=Y=#-y*_u2uIkAg~a3|xOR!FLd3!~6WpVB?xMgC z0DGj5E{|Au>vg!D0&TqDr&$|`5n?wBmbq7_@)+9yp79+@Bn}8V5zlX>lJzGy8NjGy z51Ysp*b*JjZM;uokvb(=*Gf&`9Yr^#6?zFNU4@1%E>o$%WgEsZ&y!+AS=54pzYcuA zrZg91*j^k;YV&zk#<=+jCDFwGXA)_=`%Jhzjk?wQsJ2S6Ak^SelBg+DGoks^c1;ig zS8VIg@*%dC%vjMTu6Rc<={>~H(PeNsL9imRq>}W|r93#CEQBF2nnKyfclBlDlav;> zvPchY**Ih?r%=D-IK=9R!GHAesU1-dQ3|mtf}w<;NSX2Oo__u3eRbIs4GPCi>vsb` zcjIQ9W8^+gMZcHn1>JdQMtqD>!pTg3ZB8^3?kO1PKeQ5-z@2aLM4iwf*pmob=^ZY& zk9f3m>({+!Y{$RKr@lr%)3e|euIlhX$LPQJvqWiovfA-z3`=P|L{)`WICuLi(3gbC zUOD8U*o_A?4-<52EzUy;V6LbzpV+uy>f`A?TK*)0fonZ{$r^wd#v5?_U>*{0cABE7 zsIDb?d-3<`A?ihf%v=+jc`kfzgtzW!`i+f)7;_sZbNY}tCOJaKcs5us2b*~CRT@H{C+=7<$>w;jJrmxGE?Q*H>X!)1wF0%&g;XOSS8e)s zeMfIJvE?h{U9;LR4+}TZr=7@pAMDtRBZ{i1TsZiCKlFiMMRu+1j(?jpco`%_$lW#haUkbaJ_2&JC80$GJ|3^ zr5k-wDuR9?Frs#*2|hdn_6jpEM~ag+(QeUTepOcMXTE7G>6HW32angTe07`M>D(W5 zT`zIj;2VqZFV+Y#${+Mi_6rf%Ts-V9QFk_ui_W)f!GGc3plM%1MOMovSsb*H1 zvVyOj47D5HA6;7SBHk!(7WhOsOdC3=GiXn1HmT!41NoTRLzmw#RxBs!Ko%pFnQxqR zb%TmaFMFYmGs(!nQqausN+yG>l?wab4sXGiEY!)W!bq=Gec7aBJT|X%y!RRmuDG&V zX1{hz`K4uBcsbnQMWYc#US{W<8|teLq*b&IJ==0t%o4}DSLL0-WW!yXDqd#F9z9T6 zBSOO1H;;7AaFC^F&WY~t7qxg@XLG~2V<=1|y~h*XS6#`l3$){a9Xh)cBXG%`0UoL% zb~pwq#RS7M1okSQjn;sxhG@X$T|1LXxz%nl8;itOb2v%^DEpm&8$;ti+jgzve56zsR!i93=NO$v z_OxQQt4?q0_65Gp8yy1(DZgqMbhguWWaWP%rjkKj+4csScI%X6(aDRxB#?ZyM^TP; zFnjIeuwE16r1=I5=WJ(#jx#91nzJ!iORM9#;iO&N35sFa;#Qs_ws}yiU>#o-9 zkAYk|fiFwF@jNvzN#>V|W-N7j3|(4zq;Znm@MEUu*S*}qJ8^g463afV9jhVgy3fJM zKu5;6xZ0-}m~cdtA~TPqLo1p_+L!RpFJFf_9i^ zG^Zf=qlOV%m>(2(FQ|pLGepS69{8aK=T|M{*4W6jBYb18rcT|ZGJp+(!ZYCQbJx46 zh%x7lg(5ug!`@o3mlirF`8bbvzXe&;EMUI`$Z&b$70RU;Z6_Me%Mx!b)6n_t(n082 z1;5}hJ6@0ag0@P!1C007OCYVBPQsY&K1ERqzx9|Ir`8BXF>0s}~zypm; z`xu>29ja@XvP9&Q_U=l&xF3+wd-aXh+YKx?VBNlztb%RI-mI7?xQu~%p|! z$998*w6uE6_>Y+@_M`5i&V%0`Qx_e%L;y?-bqu!hWMJmUX6C1 zO!gM`rx}+bUw66hakLcm?OIHY&dwA4$jW+^&*=31k|^rGJls0eO$V81$*F>88hJ(czaPjOrj4RsheIs>VgR90%Ve<-;~JZe`moJk_V;Yc*IN&hFO@7yB_9W(tg6VC_GPhW&-CeQ3!nO>;>bvvxvP^M zC(7V_10s-z5qI!W1E3P-g z9aof2y!y^6S-+F;cZ@#4x|^rNF*YmshoRb?;1VQ4r+lpNEpUn+m+g7J9edXCvgJnG zO{mrkO~H6+4aaRpjz|P^QrM*r%yl}67L*WjUvrYw)mq6a+EHSSkb`xi9l89~*`Di> z5s=Q`EgUS+{njv;M-7|vw008L2MS*SB2g7Agt74z%@QqUZ0(b@k!Y1l+w6VB8(egk zwkhyeh5w|CQgUygMxs}?JD}66=;s%UhCG%@a2U(bGVT-#{&VRcEn=H!!B`q0#YrED zz)vlv(F`D7W`GIIbF*p-zl3=ZqKekhzFM=^-RpWZX7W(^Bg6iNB7?X*AoW&e`be1efA119#&O33L4O?Gc zvCZ>Gou55TmqdC#Q@3Qp%j*pL|2X}>6aS}}LI+0Bqh4*;otNHlXY*X>o^gddS@0@> zREK`RE2rb~wYN_LGc|sf#!BD$fB!=z$5P0` zPT+PoqR??@p}nyk{B}ut*y`}!1oy;sc)#uKu=_1?;y->k{}sW1ayv$mg3&_a#*<0Y z@NWhzCX(9;Nc`a$M$QJ9Z;42QU!#ss^M?z_gFTp~q{vSm<@0HBExe>anxM9%Rzg6E zfvGksJ(;fesbzYzxqUp8;<38!17D+jGPhXKp#78ckDenj1S+OVUB4?SO3*22E-HhJ2&V~m zu}h#VYm{VWS2a8W5*KW|vXX(5kCT3r2zoSf?y=bOO^zI3O=Ru#TwpO&bt2a6lctX& zFWbC!O2NqJ>5|Ou%xPz}SK)o^d-9P9zdX<31h&eiNupwJc#q!ma7xvcO$ht?gu#$z z9X|v~-g{X%9Z>l+V{3prJuCHM%-C@NcLBRRPA@JAb+3tmR_Z@Bf2LXMVl?q$yB4tZolve)mSH{pu$YZ|4}0@CT^-jxj7I zZxg2y;MaJo!D!zp|sYjc;p_ko?m-5c3=MKyVBvU|3i&foJZA&SJ3BaHDf7i=qM?uZRb434g{%y z-&ISpO^l0lGisFEnUbB)lnKuYy=XyOuwI*@tD^xz62D9p13`rwG#4u(s?*w<62BO~ zz(gc$+hwdF4n%{tr_V1Ur18|raqGF!hpZgM?t>F_&Dxq3XbJye_|h`MgxBZaeA_{y z#ZztMw7DBlX~N7~&2u5Su=KNeMn9W1B_I1PgcA;Sq2ed?_v+!d^U_AncxI%39d-YL z4c%MT1OAZCU)-aE-FrRMkt#D8xG-xIY1zTwsQf?aEfa$!I+^K4E18Ru@ifj5UksUL ri8JzhuydY0&*s1idgcFH(79j4)f+)e$3Xi}IM|eGi?)!}! ze@8uZXJ=()Wmei(*&Qk;Bl-mn7Y+mj!J( zJ;=O)_H~FJI4tT>0Li~$g6!F}i5M<6gVKQ@CI%1bh9IVKf!bpIK1`II8tt?6(spE} z)D&TA@%D1}L1iu@2!#jo9cCXW1I10SAAfXYa08hFLVN_GVafEwtQEzi?aLN&9g8EA zx>t!4m9@84=O`GkAx{YI%dcMpAr2C=mE$IY$qic}l#^#QxM%0~-6Q+7U&h2#q@CpRaHL1ta^j^rWdH`abz{irq$dSs|@4;ge| zXci-{VGcW+rGCZuZ@H`>LXv?uJ5(N?0XLmwAb@X9S}Q&avwqqqwTUAu7K5ad9qq0A zfHPLjT)f+L_x0jCj8$W=`bW2!ii9 zW}{pTYf|6%t3D{K7#NovgatqR5x8MJpXD}))^wDY05J@tydIfefKd!0gx;5BkVaom zIT#mz-fS>@h=y!Xe8?dIJdQ7|J!E!Y)5`)8pMeR*MBgFecBY; zkCK5vCCFJ{aT(Jk=xtjL2$>F1^|Nk=2MDc&xYRAWtivi0=C=p1FmC46y2?~ix8m2%%6zXGO7{rE-Kq5!z8!R8R>P6S5 zV)&A*G7dRTJC1AucJoDYpj0oi`d*Ek4K*!LJ78icQU9d6s2Z!rwffug0|&kgVo`8i z&(n6U9cA-fGs(KQ7JMZyhxzL1(d+TsqvK=RBg(5NSW31` zH8Bt+7nVnmE+@h+*)Gv8o|2nmw=B`u%~@ zwnV9vZXpO)dLsWkn@wR#F-wk5VwzNnG#M2?bOJR2wE~r$Oh+7*%$Zc1OjW`}GEqD; zrk(CdA?$kr^*-G|+$yOu<-%9i@2r&4@oY(mzhsi=;s7*W!tPQ~^m){Klzp)y@g)hI z^vqI1W*RK6xM__EE+HDqZKSKW+<2 z77iA47L*nR7N!e>3+8`h{%GAP+&TW4!l^}26~WC*xA$Yr$;sWx`#N8|c(jx}J3ogo zH@;vs<5?~|Te~1JPqpx8CUsh632}Nedn5miKe*ViEWW%>wMg|apfb}Y@SJUfZG({) zlDAN&Q73-$n^$RP-Mhsb?%Ns^2e@W{QotJ&JA%Ftb)Qk6MBj->b!-F5l3CYGH_0hU zn{c}@YdBOSYs!79H*+16Br~_s{j}EL`V!Al z``l|}P!8l;?(A+O*~09YL!EKk{>gwyI2tCAn&(uQ35H(7MCs6w6d>3rZ+$K`}ksPwVsMIK$ zAq&x1v_8toV^NCOx}VHrErU-b;fe4=1(U|}3*D>(cOfU#?9A+bYm`k$=lAD$8*Mzu zNVd^n(cZE+Qw&pZQxrO>I^sIgI%u0Jn!^dEl=#V%V`F!)=gTt2c67uWC_hz0> zKqu|n-wspz9*NQYY6HKA*0pn*49>!+sS{KRD|pbTamK=UQGz4GL%8uDf-r_l^`!@y z_sVuDmd{T+POZYe?;BMi|XqXMwej4(ivEmE2AKFI)o@`+B3n z^WF2$^T+ie^@&b`?pp59vOkkX+o~n37gm+QJ->Mabn92W?=lzA!1x5Tp?=knyfOZ~&y`a=Op%dSj; zrmY4V2MxLgkN_g2ssh7+5SGfz>Mn2X> z$_0!?uZqKka9n>pI+j%yHbNCnt#t@?*tEgue*6~u`8_j_I(tXwa4pFlMms6s>C==w z-xlw=Yx`l~S?EDL7e$%Oj0{G-V8w|B#cJHzk2Co*aDq+(-VDTyB5%g0g7^Bt<5A{X z)3B+y$%BjxK$4eDZTQw*Eq94WGezU!&eQNs6Fp7aw9m}N?!Kf>(&R(yeP-J!@aY!g zg_=Osx~fsTu5)qKbiwtHYMHL=o#sQ6PE*UKci*|lVRxKg9eakY%hq+X^D*1aeFL}T z1;YB5HV<0{6FH}7N83~H)wd<2T0$w_akoG(rAxIx_A6)!6s%o1uZtwLm>D?N*nd25 zuWxr4JRS!waJqbOMm$8kRM<)Zs~jebF7E4=uY))HQ=&STx>6nEA41R3rK-)ggf#x$_-Pn34uP`y}BrQ^S9Bsy*Rf(m%sbMMJzjml0KgfPlf9E2%lD zNlS4V+FH};8`&Bd(*dpRK66_T5FQ}c=UZ!ICw*d|wUvz{7m%0aZw#)__kV%uNr?YO zakAtkQInP<7P56PCT68$p<^Hcz!4J@^Eeopa4Gx{`S0nUpLj{koSf{q=;>WuUFlqz z>1-WL=@~gWIq4ag=$V*kKQU+>-E5rnfwVS`-~Nlp|Dp54*wN6z+|J3|)`s{my7~sT z&Q81}B!4aRKd=Ad)5+ZA|E^@?_}|<5+#vm5-_SGCG0^`H?a!$^e?hqvj2&&Qoc|)P zY-8>OVB+~3`G14|?`r=wOUTyR&cWEx@pBr0@xQ116ZXIHWgX0oKlk_7GXI1BPuTy) zmou|^K}Nz3YWU654`Ed`Pt;;tBcD+hnulNjGGol zeTXm2zXm@u@+C_jrxP3sGU&gC9Ynrw@wb2B`ogf_fl?Ar{M4X8{Wo$9ST2AVjO3qV z+Uk=43!X)W7{m|$e-5~i=l{(^9ZC$?tW~iy$-g*2{#@%1_`iq}gH4F=!->`9EQbD@ z=wHkt{fF8Ae@Zs%t5}V!M1mPLj22Pm&+=0YAY&za8HlRc zDS`FR6Y7U4(@ja7(h0)U{xxd4u2DfS^*%1v94_T&@$ZU?TLG8C!hy2cf0B(0_eH|+ z*6Q4IN@sVXgQI%89K|RTugaz*#OWk9!L0AS;M)W(8=H;Y{W3~WCQez#@Q<_WXY{rE z>l6Cz=y!pvdoq5`GFo`CIUR0hx5p>GjVe#V%4FfdG$UAZjd3dJf0za_CkzKx`E@o{ z@%Bdop*nx+V`~aze>AcLRfhssYH~#w+CTOr1^|-*y-Hrg2;QG~lb}sv~ zl-8`5_$MN=U;ejjH~zTuxRTy~8BNG$>98|I#akr{;llrEr~KEb zF<^1+OERqgmTdkvVo1_sx)ds!f7$BZeyXQrQw~_$PW7GuHJh*6{R=5AgkggSb(Nf~toL7u+eCa@(+O_`&A@>6zuJluFitZ4x;UP zZfKwibpL7w;Txm&f+639pi=yYF8Cqh0(~8OM)UO2vEnY21*=ka?j+vCz8hlD{e$!A zU04oC(jb5Pu%>|})??~o{@eVnK#PCMt(zRx4o(bX51nRHDE!~P*B6E~&i4RRoK!o3 z_8&w)y(qHqrm6Gqm-O1RueU_0MOv{H4{rlxIxh`rU?J4_ki9^LJI0;@ z8rd{%%Fn)fugq4O=N~W1$U1v`Tk{vcJq?*x_xa3yXdQA)O!`Z({^vxQSw4#2p*)ruz0*+-f#EE5oQ% z)2q0-(XLDqH$5+_=-P!V3QEl|2{Rp4TKB`|KCik1t(zExxl*2U|4@NxqJ0*`nsW;( znrRWf9-*qhnL=d{tmxFA@|=+P@>2GBg%kP#FtaXla-hHE&8W(6^H_cl-R|PQHk059 zqvKP57vtQS`FWs79V0Q2UpqQyAW|iS?k(a%+3ogrn60Q`9Iu^pKsOw?l*N-pBKnK4TCgZ1#Tb8fh`I8d9ZwG#D?`FF#pldMdEHS5cUMS1{Ia(Cm@4y271ckFmU3>jl z+OC#97U?c3E77I@SMMUwY>_`gcP3RKAB(a+tU31F1pm06N?E@6WUCROE*eM#W0#*$ zNv8t&LW_eHzX3c}|DXbDfF`{+!xN@X!|P}`$62GVNG|2+#UeoyHG^I7UXjW5-o;%G zgx1fc4cc1!Zs|5Kzu#HdQhl{5YRM+s*)0d%_g-%pQhK%TCxw7h(BgAyf=D}Nb`{y= z6ZNX$-b`JiEujUktiaF4o~}gi!heU-tMd@z0#C7JQGH2e zk^|OlrQY>9rV94}WKeSY<%~CCM1tfB1`00~fBB!SRrTUHIX^#jBVFF@=K=q`M)tHEnQt19-{1AEVKi+3-@VRcxez09DsUKNR9qtnRM_n3rII7_&yL zwB;YkOv$MP{TZrngJXR&*9J)K03AHIUGj)ES2PWIuyy+dC8r~WNe(b18XeLV%##~T zBOAEE%(FyrmCJwC>88Y5iDk1I6%3s@0c@O54ZJgOpUGI5VmkGU_1v zm@?qf>M>RMb#0Zl0!DilCP3vYp3)4-Uln2~2{B}T=&;7Nj*L6uYoHX~3GJQex4g8ch)kd(f#Ql1QXDS+=!WR&+r!DR8xDy(9dDieqQPDPj2Sm5G_uZl z>5V?QSXg*{qfY_9^ddEDB^(*~g##P-LvXX{pTOKQQ04#hl?>4iCNq|y3>O%jHG$+F z4VX%Z>BG6-j}DcH(&r1BlQ;d=Ebmff^tR~AY*}vc0)U_QU7<8+8cf+K#?A2*G&`@A z%ujoYb2b}I`O2jbi6Ux*<#|jA#5@OO^&FhgwiAIXWg5QLZdR`T9gfo^vV%fp(^k_b zfILL&1%3#pYi^)irV7=DCZuQ}X$Io%MnX;dgnQM;ixxv4Zs-Hbg?@#Vy-JWow-H4lQ(Reo0e7*ls zP$ud4-RV3~`c^hZqO?gGq)AuiM2Rlw%6&E-D1G=HjE>uokZF~XKTXiRmUai%W2Pr< zYH&7AD3U{kI(;Du{7Vu-s^2##8t0U})ITko_~Y1!BC?`LT)&mJ-x=de7*;`q%M*WE zqRHo{<=Jor!-RA%N`DsjXxctVqQ6F|YZ=O_!84Z+Jn-io8zrYK1hrp>yo6*vZ=R#X zohOaDvw0lwz-V$_nr8%b$vQ{Q0y!Z$#R^Ci1eOrkKcIr94_?~rJ9;1t(TKqd@K-9z zyR8k$cUyRAu+zg}*J8zbRtg@Ber7-yB4)Qi(|&b2ft9>o~y=*jXI%!f8d~e*SGRsgGC=;_XE+A{nk|p1?1ZeTPvpqLV2$5|tp}9pj{eUWs zhY8^yBi#=2S*E3}vlvmPR%CA9BwqYw^ewvZIjQjlrv)G9)lC@-i2Y5EzBy7}c-)$- zUrMB-_L_l|8w(zutM$f8N_|AQ-gt3Q+Z4`pV8A?Ca|VjyUL8!=qWiaNzFeaDV)WP2 z#Fx`(SP2zAyq)!!S<7#EE>baow>77cJ~{&U%Zn#ot(fMPJorJK3EOk6Le@@-~Vuk6W{T%?|6xR3u9LY(PkuJV=^FB>x1$dK*$m+KKy?nHkynJ^Q#PnDaOW-aJ5{y^YC=p1SurF_#U_R~KDisC zPWDy2Mhg~rbXsF0w9E-J?pu(QUy4W*Xr`>>4I0Bap$VtEsSNa~Rx45^HWzr_ zDthx1cxoP7;rMeIRcv2pg853hnHTfd5li_7^8LAq+_*$Y_^7I?;APtb1BDU(J8*Zp z-W!i6`#(i%bqJ=1;$n(Y;@6jiZZ0B7W`i|49x;<`^Y~?Iu`GRR`#fln zJ&Md83Z-O@a1JB8ItS$VM&!kgQ5nJBZ#VKa{(eVFaP0NV_6Y-&M;mP*rbJE153gPf z3bV^saaJpKl6_wsYIGtpn%#$;+5t`6gZAFY^cP);&j>COXta)=a@#tc!3xXtoKk{S zGd~+SM(FZOKZ9$G(_gm{S7dcq3UG&-K$lH^1lx>YZc&*&hIihFq?+$#(80NM#zVj3~%%e`4P4!5WKOJ>_nJb}pet+U$Y zhr(COiE_F*5%#rwsF^%vSdAB|Y3e)#3Pg=@>jLaFZ3rKzpdP{aV0F-UqN%gq(nGDB zBLzo^u9jr}rrMV-JOrZXrpLq|G=-RMJPD5u_7t8i^#O|6aRUa`fapK)y%7d%jxJ1; zMCADzh4oPzV|IP&bD%P&d$zvUDEiy?1*S)lrqX*5K^@heDOqTYsTicSLy0#_R^j(n z4^aNiK274w`xU(D%KaN#wACp(i|qWCg9O~N11{F8KkM$~&FBA)F0?_uTGgm_WT(c) zD5F_b=ONxWkQ@zL5~tH9WbaM6-$tpJIGG5t*X#`gxudkyBgEaf66dsc8wh;5libd# zvwidI#DAa1g3PBih@qu+)H5v?)N_D5Hw!h~dbJ$%Bz|7{(ESK9H zr2fiO6n`i~_q-K%rK&`bkd4`0{gE_^Vff(RqW6UvtGv^x#{k*8;{iIlc8iu;t4mG^h%VPv9MNE<5?pn{ppXTGPq9babA|i$%VC3ZQ=9_HY%-2Q5v7!$uZuK z|Fq)AbpX&Y*#2n69Yb`d*OAl8IYv_~K)Tvhu&84r?aI(b?I6#Z8PfxM`OJ7r>zeBD$iP>pcxN_{Bk4xFiX@Gan6eLu});TwUUHcXM>+nnW;r9v$3%ba_h*~ zb%77HzUs*~5`RV^6JOr~0F#<>A1QI1X;FoxAgC8WE0S1p?RW3u6#f}+-*~PRMJ<~) zglIm=r+ID=);geIBq#414+yy@e#XJ&dJ@;Q`5&2mj zLg=6r1x`EvX~XccKTPjb)A-$E)=J%8x5wdRKH!w(1Zp-u*3(jvY+S$uo+L#WWUYzC zw0{B7dD|ygEYgAsWQ@-?Lt{QQ91T$vTmJf*vagn&74-&K#;BtG15XcG_)PN^)BAj< z_}I7)b9(E?VgqUOX=G)tI<277{^@KR1Xemnllx}$qoBnEJ*#f6tT}T*z?^xy^KY$N zBCQZ))0ItZO9`hETGxs4rNl{>2b(eby<*5>D#^W9di!k_L*`yO1HxM?&A_(Clp_;1 z+j2>5W);<<&wJeE(Z8+U}II5k08cv~~V(uNHEysoDFteB<49N&7i?1(7H;i2ZPnmGF|4@RZX6WwYw ziwt>gCT{IAwR9PGmo#?BMbz-F5>@^hlzMMu@qb95ROY2^51&Q32lcGV_|NUw++c|lPkX-YV*m6*gG@s=LoaH)gW|U zItHl`POu>jAWbOHIMd~kCW26r-)|jC$l$aY(prM?gPGjsajX%|oY9DZ5Bvr12HzGA zJu#O6KhQnK(Hmh{L$Zu4VN|4DGBp>GO6aeL9=MX9e$+73y7&FQwCQ2Nl=^vx-tk4R zFj6VOWcO!Io&9F_){*Fj9Tt@-|o`q_29Bmzu?f@>NgY9;$v;t6(J6HJ_Q66-`SdAc}-2z++2& z5sO~ITur0Lc@oQEYMK*!L*FfL_9j(%z(hq?Wtp&P1_Ry`Ws}#4t<}V_5~an!`W9m~ z7s<#~(vT9=Igl(e*M^g~N7S{HKZnTUyc4bne-hgdOrv>xfNIbn9-ajLacT(s zYNA;R$i%TEr2?}@QMj}J0M)#;E6sLP14K3H=W@$(!>%JR6oA8lM?K_Wl*#y&_FDfV zje3!@!*n~uZ!G0`ap}k{ldW*a<${YWF0shRiZ4Y42~q4SD#}%MEj!`wM-^t2f2EZ- z33fGv2ohB=`8z330aw@8R==5#*eNi8jzw2T=?St#uRn+EIhN>J7uq|@*>yBlF*jB* zcX#A(!5eF=slIFcAPxg*3w#P!RUY!`#}vqmnjXdE?;dzzh+X)|;BMMXX0D4%=|$G_ zTmh#?*y&6kjIxELSY*;4xdjL{<+2Od)A+ts-L<*;Bpn zBr=glP}%t`MN^(r)epx}e{By{;Sq0pDl{>|cuT!tNC_%@q<@xrO=Wn1l(w8dbYPn3 zpSG5^az(+UY7`-i90RvH-(;!oN@u+wDZOS zzj_SJH}CFh_IK+e;-Gi??G?l8C3ppqy1ptcS~o9xIfc}CH#HFwlBU4!fd8~tSt&Ay z+0xUp;nIQ=yu^8gnhfRG?`-5clp#gIag0%P-}egDy-Dk$Hl_Z^F{|ZfZ<9lKmFH^B zXx3))ex>a6B^DW!`_3BBU_A}oj@@M#%CGif=5?QCsUvDk;7oeGWyaVhXd;fKfW9eD zV>L86h9msk5#fw8D;+)qfpTOr1l?{J>$fJn^US$;e`Dk(q_j|$G}C^_f|E!^_&1do zR{ZimLvDSCeJzpwk)V1$IVK9iw%)WO(s~*GDgGyu*}^0V=YsYM$6Nww0>{FT>dg;V z=T;nfW!6S~*N=8BF4UJxMIOYi&YsjED0QZv*=$p~sD64O2-X_YJqRUy+~Eg#u_5T6ac$VwzGj^~O>f&kqWB-e@a$d!Wo+J_z6Z$HBRUp+ZFT`Pag>ll-xfnt?Q6f{bwG2zdcqr5;` zcFAy6m`kXoK$;vJY%jgKpp$UQN|zH@MolLrN_Fq@2nu_jHZ4iuw;rMr(|ex=*Ll~2 zzTdseNoB?foQTGpE1qzLwkWu11Ti>=5y^HjBb$xc7=_Ln(3m*G&cM|^(1qL2#)iu) zpB53uVe9h@9;X>8N?V6+->Gvx_Ei~T?@Z*LjL=I3vr0UCUGLf;Z7q>saSRiPS1Kw* zP7Cb_yZCt%JkjHuSueaW6Je@TknTpNk6%|k|J+`udck_hTIBO5U6LQ(-;9$347g>+Iyy}LDQ#^f0TA`Vm7P|@IsX`tTf!%$= z_cn`@eYnHXqc2OxkMB#)8sr#K7JbP~>2)Nr9%wZ=0;gX{B9cTFP;aV)qvTKu3fqxZ z#$AbBLi%L*u*{=+EcsgBOIeZ$wOam)YEsO5xs%#%7yvKV@YBox3g?9OUW_SR4SM@& zNpyQ@q=fFi&qT1QExC4Q1VL}`l%@T0(aaRG#Nd*Z$oPbI^0~K#rfJItQU+j!V3*1>^Zl@H)i$w!zB=|&+CyBhmKSoy@x zOIiVJ@!|A2wRa~k+o;zt#s(L&cr_refQaD!3mp+dY$YDL;NJS3J~gf!Kj_I$i4y-S z0)NyuBQnauio(@%b)$-cv1wzhNe8G%AebY@OIQ;|yaA0xum{okIHE!?V;u17gr`zU zqkw8AG~r3$faL16nxfum#0kB!30w|?_dR7@Y_C%%=sDIHSlxSUnvdieCg+K?1qgGJ z=}tRQBV_r6y;jXB#~NQk?9RhEOXz3{1wMFPsB|rMyj%+r1yX^g=7k$foMk&F-)ZL^ z!{S~BS;y7*txm+twnt&&qI{cmni?&^-yK_c!reR?4J;_2XdnUgo~H}mtc0sBY&Sh~ zPl;8Q9ykd%L2@Fxi3U`7ke;yg)OUKI%yTpcJj7l5O+#@Ct;~-S3~ApIr|2GXMKxgF zR#kR0^%dsd5;3Yyj@k0pU;CSM3@DgnT721-P{~f+o~B$)w89tTTCd)hOzDfJ zfLcl=D9$Tmp8Y;L9Wv1##bX@@UQsFP(c)XK9I~3+FiNRKpFP+il`6~*@Rm) zAm~XFNjKj3-t%y$U2~`qn01+Kif$2P{qWoSVo#2;NPW8l_1;EF3VxA!1=o+C$t0}7 z$gEVWWn{ngl6R0C>{y6=b&*`-NGD&FC^_X*+e%ts`VWDB~W^CUW_@!KXUl`SE74nW}K^> z+Xi7ZO39#!4ppmfiqyQN-&xg-iZTb5YtAH`B&mD6y_WR&EeK!DPfexTaPxKD+xeBt z%poBs=f{z~$4?@LOmB1QQy;e-4+&KST&yHhvoL!g3?%y(7;@^<#RR5ak3E0y)zvwYEQ$$t&G2ojmt9HPtyXqJpSJf%Bw-9?aemJEZK&!eRTp7CKzlaJ) zF+hz}W6nw|ql7E9TN_}$?22w94Z7BU3u@45QhJ!ECQ%S3^Ws1Rc;YY(kF^xE96q;h zoTUZ7;VO;W;IP70O`Q1~KPjCTL$p5pxUIy~a&*@BkL$9!p>8e0%8m<@oTmhGTCXDsRv`Nlb;B|{-OdGb^ZP2N` zk>D6F%g^n<6f~T#qep9jtUZgmH^&D%S9_JIbrJJjmB*eCZB*0Vx7S-*hbL^rSf1~{ z?+EzWZLs$f&n}|B^cH+wx6-9QG4avvK%I6`O)+^Eg%gYSHxb;J&(xfWQnEgd=S@6E;>CXC&fy3lf%DHufxF3I^}fv+>s{#(Y&PFE$Sn?1vpf;CkQtg~CyKBJpax zjNnCL3YF|)q5Ko=Wqd$~K$SQFtpW0TvNE46s!)HvQ)afR z3|wt{!X@Nb>}X2zo73S*jQj{oP9cna4c|Pkzp#FeUBIZQ2+Ym0ZS<29Gz86i@*7jE z3hYVN0U=wfBw!Mcg(%obdxx+alS9+1NsU_L$l!q?Mn=TtgBMVTwW&FsG7ZxXNfWC+ z70uxU>w@2bpX1J?8-xeO^STT|qqbeZozN2aQ@8z&b-)N*R|O-H{>%22D)i3O62lxu z(3x8Ji6mtAa1OQIC`$O%t7OM^9!5W0+~ANXvEI%Nr}Yk{W-&EHigdb(13Qw=bu#VA<%0H_+#5DJRU` zv31KFNDE82TSVGjV9Pzhp<~tE-TT+b5S?Vo3Q4GNMO$)^Dq2YCRh4eHP;(lHcvv(? zI0wzc{3*a#Pg?R50-SE+zZwp9Dh)1Q2S#$5manGl>fga&Bvd;+Q@7n;;UA}?g^u_c zP-3u-f<%{$z{SQ*s(roAxgu$%?7Ft^UhPgVkIatejo3yP6q?jBiJ`wwLOz5zYS%qcryyOQA2hMJ>d+_ zCmm<9@HDV|k-cQ5;`TjTQ#E+8(ok`jvzN(IK(K0nDgvsg(+{-oI<;CGZ&&OST;7jH zsVU7TvfFW4fv_axjZ#b2+MnDC0QFu*IEt1$U%eKQ^BM4>+Nx5k&omX~Wq_5Jx!i`- z0KRQWgz=o{(4f2d&gxJogAD79I!;eE=F+PcBd^g?gO#N}iae*p9(qM4@=xDwBE9(5 z!6g?>n7qCdMaZDQV6Dx6aQTIGo(3bS{bav?j*6BC9G!S%u{uMh=JJ3`x$qYA6Np3J z9%ME4SffGR81qYtfvs2<{~;$`4g_Yio;2PRws;@!GAy3ZxSTzm=G+->zD@#f`8K{U zTn}s_pNKk>rRNHU|KZa;_hpVg_XpKZ$_x8;Ls>s6LaXKMob7)dJXg3E^x^F4%mRK@ z2!|klXugZl)?b5ZD!t9?u=UIDQM;;OIcLKG;K<2@(+6k3h+8;KwCoI!IbR~6m$?u> z`OzRuqO#-Mm||c^*N%f9;qGKAqadk2Fl*pps3pt&%K*EB!K?X^q7VV`^1X-&^+YFX zsgbK_i3%3Gj0(=1psjuYcIYfjl<>v(K|BMH*`L2isfzxy!~AGI4}6l(!^iXH&B^oq z9ay-DNR6hP09t?XE8wJ(iy%+r-UKWV`4IP}4YY|Ns+x6I_v0OmzVnf6KcZTpT7cyT z4o!Xv+dy!m_S#Ow-2GtgfaN7o;?9VKV{Zf6^~7~QSHz|_NM*|_%=@-37|#q zaiy~JICtM8-nE87R{cnU_|c6u$6Q)5Cw~oki*NVYkx%CPv)FC;5*4B_^`+tP?m!VVm}iH^oiX>s-JhFeY7$FPUE@Ns7T|S4NJS-N7g=cr)HrMVf)0y?)e%5P;!KK+nsKgv%-u<8h zU%$Wxz^9lO*aR{=Y$hkn4mLwb5m$p;q_YXMyKX-yLEXr!^rOJkBFe*@J@9^lW;mv=Z;A`J3@3<%;_-b$-9u2~#{U zyggcgnFhm@h5e$y4;fh$-NA`Yjq|coz*L)asC+iM+WAX0rfD_+I+I8Qw0pB}9ow)* zEoQVAJpw;ATvA58n`;(TQ&m3He*It_y^U-Kx5ZLwVE6hekHcx4M%D936^Xe*)wrqy zz1Wbi`3+cat1UAo+gFi)YJ6%0SZ8r3hpu0gO9`zwS{A3K?F?MHSv>xB+1GnNsemT= zRPA+Qnf6v$fC=jQ=6@vacNKSoB-rLM&W{XJ`w{0FM=#@BND?b8@KW!HI8cu*0M}kSDVYf6?v=4{&>sZN8YCv%7kNu0>F6cp{q872fX~i+%Tbzae>L zItQ36;)O_QV9RjoBaJ_u>1zRx_0Ln6&CeWIT|&PTtypXB5Hx<2jv)^94wNyT52bMY z?zYB{_Y?B4pUenLJs~VnwN?VW%z1>G@X>`=yCp+f^3w z29W~gkMz|#gbW;ThnaenW*>bh8+;`xOBf2;@tfWH^HZz!splftzORK(4QBCDgsyT$ zsQB~7R4PgCS1YA1KsR%W_O@I!?+;B69#$)i$|F7`R~a0ccw^!5*qXrYWq&wUQ2HPH zw)D^K(H3oR`;e4Uw&JjyBZ<`s`EWtXIL*6j*427sx~2Fj)_SfRM~ka<%LXPc$MtXo zOp@}Qi+!M37H1~-8F5yWAP4#=EL4*zZHPTnw)xF zEANxL?61U2e~Q+=1;vLiZ%VogTIrB`orAPC(P`w-{bzaPB>Yv#?&jr9$*B4&5s2wa z$CEX4_V22BD^_FvYFjwt6=Un@9oQ?*w3RpmoP0-aD5@#Hw-d4!qmBoxuet-@SuAH$ zm1UpqvfMhz&A|O2i=BQY%$+R^rIn(9xr7G~&AcE)&0o~+5x0NrK=8a(P1%oq8LH!3 z>HGHVW@V!>XG+%tiNz}sxwIbqhprt|;i%R4T=a{6zoW;9^QfdbK&)j%c8|?z{U|3dhaE%Wm`DMZf09OR9|lVg2as&L2yeZoYFq=$4is;C>MO zQR_mt<5Fp>V?{3i-E5tz-bAV(zRP^pv9z&`PIpEVn5vE|SfGC<8=mz0_x>(S?t?_Yt0?&R+yJ#N!G%4e+C@SJV&LJ+|M>6dACz*O{}M`aIQ zI)ddZAs=yCeDE1N09yCJ(JB2Ly#1oV3ITLaIkBYUoR#r#iHE_7egFM~ z?e_YVmh~!n5;a?2R5d>k?20W5#S^^iG@!T zyrv!>O{0`s8--z(vvsvy*sj-OW=yg2wRlUl+uH5qT!GOYSVH^@fC+m$qX^ZN^$J43j!WBEV8;G46 zM7Az?t3-@#KB$W0s#ZJd@W^U*~KTfPN7-*fD} z25&HrZ?&?nEvCXrQ@@^Ns@7P!5z=uSf)ph`miQ{oELaQ|Z*pX<*)(720&n;(@_5L= zAt43U-RIDN9|5pRg;4=r($u4lCX9RB(SX^dj~(px_eBmWBBI(?d@70+v564@oC>id ztvgagJ|}Lc^SP8oCk@&1ueYciCl4>#UN2VoqGYC%5s<}~M*8>TPaMaAYwCwl&9A#c z>Q$(l4?Lo%v?I!E`JkEuPk`}~49SfmWuRrp$7+Y+gKmgDRIpQVcb-t#IyY}HW~>Z? z98r>NNs`}?^VUu5-symezd*JBH_#lV2s482@Vv0gB>}Q_#&9>JZG@VJ_FtxRQ2G_YS%CIX3f#$_oick>X8m`@=uZ zMiKRDi12(+##0S$q4nrC^dVEJ$-G(MaUg*fi4~zmRuPkuO0uRZ{66J%GDCTP>(SdqFxWnpiPsft-y@XZgTe0>6CC-4uhS} zn3vR14W!-lCG<5PnI0l`i;g!g#N4i2O}14!KVzoxS6B`r>e(0~& zuqdN9e(SAybM`w6y4OK$!-a$oNB%x;D)poD1&@l(dGv?ZerhdBZkcR$weJuHv7FwJ zW;U>S{5h)PDG~`8{#nMl4;p;-(VN-P`1NJB1zH`vlsnGYj%lE$R(JqzzLHb6o!;C# zf8Xe4$%*2J=(FjgEe!YU{-`>K&QoHTED~ZalO@vBMUgS6S%7KQ`!Xf1m%Fn=r5Y9A+K7OJ@j&CWJ9&WXYI5EQDXFxywzd+ zwa)eb0bf9%zx8{IDa<#H{03+>{AU;VUGBx{M=!zPVY#TJYhXO+ z1%lV0?qEj4&uq^q-V#jvTFMx;hhCx;B*VtzV+${)Y*<{%z@hm#yZ-tJeYOvZcGn^I%S{QSb!_`{Ek!{M{3A@y#^K6UG*^*RLDam_WTcmHk}G@_UL zUSD4x!%_UU9K|-J!-QV@o1K$|9rhT>%w}}SE3d%*hwX%muQ(3Z-FPa_yx<57ryv?u zCY-!orvkj(>=+in-L+Ro9DDM9xbX60asH*pV2=aFG9g5Tt5ZX)Ggh!Y7fj%_XPA3N zM6`=qXSI5-%`ZmY(lvAmT!YGd#w8WxxvWtZsNc|Y)he>)YtTkb>pVHa&yf@tcT-Ya zg0d1?nQdavAp1ExaWG&5y?(4+TY!T65?W`diSs2>-*u=tVfB}ATm4SKc;!D-aCj4& z;t^TG3Y8c6EI(XFZ}>0$TSqe(|Hq$ZA#WAqUQ7`*CCrYcBJ#_m`6!{w)sEW_WKpav zpL1sIdGqZbohubhR%}C^hXol_QW(QZoYxJUJ=G*UQa2ubkEGon)92_pL zfAPgH7{>UNk;8jY(J9}gYY=I3bNw|nS4$HTFu~97qiK%mSxsRuQE*chdpFx?qTNt* zH^(4mw>gHcS!LYDWC;ygoFc(Fi@M}ISc&+Vu)e;a|ce4+l%av{vIvj+&Aw|;^4eMt?nzRq@ zHXjoX$-}-M?}7LRFb6Jdtw_Oz`{Mrl8zuKOb3(k7p zh|E`Y%ZseL=ZC)>ufEd-m!CNRV+PDbFwgFl^`C!kK#_^#t~p}{o_(Vi)-bLuh5Oly zwIwYL?eQ)ShR@Jze;9^CdMrb4Ki%JKfGCCq(JUrabe+Fp=wy7js58D^*p7W>UDi2w zP6xt=d$yTOQK-}@=kctO)w|;VejSC?1sU+Ahb=1`;@Hr(ZK#~B`FAgar$ZGbbbmPn zLx=Rh!3W&HEJ<{?silEBMT0_VR(1xSeB?she)oUq%l|d6Qx*q(${0-C6;J*1LY#Tt zLpbU9-6%x%=Dj)B#hHv5)fd-ZaTK2Y_ZPT_qv>@Am8|R_eUQ$&;Bee?$7!gbGPHcz zYQB#$(36&ck3ahrr=0O1iq;mhUcQa9Ow_r+CXA0@e8%@b+=qg~QVt;bv6NYu-hTII zyi1p(f|ZOzXlVRLow8%CS{o?=Im<+cj&0G2naEc19KKfBPWkZpQ<;ErHTw787G1iw zrJK}RrwBCWS>hCo%yis*$18jfQOM%=+EyG>s%YQAz=7TH+tfv@Pms?lKCEaZ8ZxLG zKKtS~Rxv=qpNWCU$34dQ0L~|$iBv+mF|(Hm2Oee&Tfmr*^DTbH9)eB*3Afr@t%6iM zGCQWpQsxK170@Hbl6!bK*< zzYP>&lcFt~y^^MERMD!?IV&H-GARKzBHKxQl{W<o|a1Uhh^=t&Hwhr9b>1VXKPLl zMjXN*0=ny*I|F_DQ}|tCFFV#UQqfw4d0fnm zhp``nP*_j_FFh_en7GM{y$daiR!Kn!_dGTfFFi1ieThLnF&4Ad1Q~$drAHwyIde8{ zy=ypya0<>&VUBUYX4+)<8qybSKT|AXvvm*>)|B zArS5z=Ax5Dp*wDGA#4vv?M;xau@JZmz+^QXA@4@P`PCP$#^u*OgR8H7mcIP3UITs!3Vjph2`2{@m@H;5VTg~Hq7&UeXUitSQ zT&AV{550|}>P#?FC$+0PMFanF0yzcl#Y@)U)i=IpHoIP&4ke9N))XgTpL^E+P6@j5 zx@YnFOYHxdHh)M13$JH4n`n>^2YmJIRFu)p?3SBP!bKPS8G(F0v>1F=QJl-4({K~F zPoBI03m2{8OR$0Ie`B1tZVk8x>d1#I%Q1x^9K6nFLmf6Yv0|t=AD3Qx3>Gk@@N_C0 zUM2=G>sSANU8skIke|zuC9{C=Ut63K#MH%0m#yUhauqsST#o(PkcXBlVe|hZ1*8eD zf%QKd4eOXzj@A^|KxJf8`$D+0h3zP=EDDL1sYJ~5C9P!{z7jmU`*>{k-#s`mWG_!17kT<^ zVmlo_3heXqGjQFvBl&=CqJlTEUX4E7gl1+z+g$U1n7~@eG|)Zzm*D0Lrh|iltPeji zaJtx!Y{rmH>W9}q=s;JgGAbXN`h;3X%*_juS%%l%YJ-bT>V@66pM%h9_L~@15Xxf! z2P>~QZ5Ez=tv5#oGx)-#lG{XK9wz3pT-aIj9OiEf8kocz4c^U5-C@ejOz zOt^{OrQWs1>|1X^`q7Pghh3NA!!TjdjgHrmJZ%ue+^BB7^EJHv+NX@e$l?4M%nrfD zmP zsKTW2)3Dnf*E3N0F?Y7M0x?4~F>FuKC64pTd_*E>rHFbH@GLUfX2N zihBGwgz*p$|Lq(O?0cDw>N6LNZf&-9#FC1g09|v2*k5dxb1#*>S-l!4Dz8OQ#1*>!Cw(eqv%Z~7{Bg2a+ zuz`X??0Mi$IPR#OaQvy{OaCehTBoBG+L_xP4-VMmkgk%EBl(wgEM;BHZOg1tj`PM{ z;#!3^-&ZLhP0gAwhW?q%Mqs149Epf232fk;;r>H~fSt>YsN2iMHd+Vgo7<@hHdx%q zmrmrgR*auBrdyEuh5eS`hqY7j+?sQ!>n*b-3+a`Mj z9@uFLF8g{EA9gmP-E@sKel{BJb;8&Bbx4LSob%0D6`c#x=Bh9UIxatb4svoB)V++O z;;HuRiC)~x&#pCH7XJS1UmCt#AFF_X+P({rt2)yea)0 ztv4&k2Os@Hm!7cVVFBHd#WEz#{B;gT#aCg^U58=H&r>MiQ^2k)!M_$6>7K1`hv6iPyu?G^LtEWir{TUdCr`z`zrl}jpPWsJqK!tMIoIclHBl#d`SS%b$6z#X*WT=Rd=W8SodR5CMJ zzV!oU2{Y@J{pUt)Y87u^?V37XYXttw4vu; z-AY)soLwcv2?~eI#iy%!U~vgYUl=^#UZiyo(0WV);b)OO9$L`CXSxN(x;pV6i{B_N z_u<8<-S8qaA06I*DIVWpBD$o}bjy+0`Ya*lzIEx)nRsSqPyDiyDHf@4RX=~5q`~a& z&d1#fv-;Y4GA-*wh32od-Sa_azUtbi2v;08!&yIKDIZod%T(p1_2Y~0`ry?MI3kx( z!fhLV`8T#zE+ZCWhTi=m8=w6+2>a~x8$#xmZC07u3oo?K6eW60TzdR0{QbE;SiCrm zqpa(@C)7TyIlsI6rx7Q#`@%=XeI?!U{`A{OJh|g{6bQ|txPBq_>~&TXx~bhi_E#M6 z$u69b(~tf7tj0Ng=Af#cIX~=v_GeV#-6cJ7>yHDNjEkpl@{V2a8O06kwQ-KBy?K4=5v9lG7YQ%X<%AWYDokPQK^BE@KG|yu+-I+m=tnEa z?RT7qbPC=z3yZMH{R$~R>`tX+x1EPDJ6>Dla-zTn&rV)bSgkMY;31B*hp2?4P~fet ztXVt!t=r?n|Bb`UD~@D5%oxVJyiKLbk4}t%Xv<)G3*fFH!2Opk%R>?C*bpJfgndiz z``&-Q!xvvo;Y5T3oN#~b6-RLLzrLJ{uCVksuMU~sT!n3r=`R2Ej?o+tZa?`>vLP%syT*o#@=(%Y2cPv_6T zdDo3)a4tvknE|RLEY=?z`zm)vW^O6!1g6QZ@Dw$e~**A0KAa@LUn;1`~I03Lk!y_(7RBIUWi zuBv=Lu>bDIS-YQnX70ODB@n~)*WS?^K+k$-Si9(T?T$y=|i6r z&PMMy3$b7ZgGcG}*pLT8>{7M0r4lgV$ewt<^hy+P{)Ll2-+_;-5RUIP-vyq>a>sJ- zJX(+s>%0s*_E?P{mb8t^+S5|ox~134#eNj}F?>uOjyq%_z(KZZ`DDGO+gvN&HdXj^ z+yGi8+9AbaKP+}7L5r20k`=^9UuNO$@A~41y{03W&pO0Vj!bw4GgzK-z+61|LO)FT ztu+Thm=&r8#DZ)LA@>}2w^61=#nHopJp&N8{L&?nNXv0Jo97=psu#Za=xbh1Dx!3KaY_uzF^1a<3d^u$DXk;4x{mJ+8g)5%{it3D+;_j$G!!_HriN7|(;4wXnyA^IN5+{?J_djd$ygb5GfU9h<6D z)PpzX;o6wQ1k8oQ%h7W)-(K>=wpd)u8DAMF99B@G8)k~N&2!?X(z0!PKH#c0J+VUc z8ML<11bDAr#7PHRwdV}>aWb%3{Dre_Rb}9>PY!lz9W8Dp(TfwTnF;E_=ZA2Vp8@8y zu(@=#bxv%S8LN9{`$}AS+C2CCsk6&wTU}qYYFJe)9>~X}6ceeeUp09YKo1S)nik`5 zH*m#Uu}5YR_R5~YQBf*yaimncKYg5UW&Fy{xc}GwbZ^=C^w=BQwir!F`b|zSq@`f! z;O^L$BlqSvXM$*C@tF@D-_179g1Na^N>mdJL+)kk$H$*ea>A*PE>b20+emq2v6v{6 zIn8ayCiiphJCPq7@z-6iI{o6C@22786Lw=MS#;yc;f&jDC@52b=f~H&jhmRxVn{4= z*2XFyRz)WduhDsJOpYu{F;3XKg?O;5{S=BGN^fAnwG>06+aUXW>NLlVLul0wYOvC`D=N(BU z0Gv%dl`dzg7TXdk!y$)l&zO}meDU>^YQ-RYB0EnN=QR1_O((jU*Dtt??p-#CsnwPL zMb6{IFmlaXQ;5&$c6RvTJF);O#!Oq7n#K%SmFBkR$6-h8hAFg+%$+(P9tP|;g(6ms zB;kM7-6;t_)s}q%_tKzXm&5E=?U|z0O+r|$?QW>$?8m0K;)fOG$UX21oP5L~JfsQ} zf1QYnFS!^89B=>*KKLMZz{9ZL{`=yvBagu2k3WXBxodFF2}k3=hu*~`B$ z=SD}_=8r#*pj($WZh(4pO=7QYO50+Wn4DWp$aBJEuo5r6`VCIzTpYYu%q&;R=cCsJ ziC5Cq#z$t%8&KrjuH2+_w3K{D_nB-eAHO?hX9}V0zqA0D(qf-LDP@83 zBAjsSuAD5cryH?%7Kxh6+`yP8?jIQ!biv7K>3HV(&(W=0I~;n<9w^OS=@bAPRJ2ck z3RMLMB`q+yu3+AfD-}LhO(mUhOg|soegB&{=gj@ldvJf0=dD4A8KB%O?2LzC&?>IG z`Y4x`>9a2;)4D-nnKyk3WfBivh@w9fceE^LL_9^!)hkemKi_hao6zs`4}NBV zI$f_~JY2%F%on>L_=w^Vuk9xE2RT!DCR5qoO3TXSe|Q?-e)glYN|>vaO*k217D*Q; zWQABosFb}&7qZ!Nmf_q;m_2nNpM^F@2wljiWZ1-! zM;^Z?cHMa}PCw@#oWzmVB9>?IAR%`iPD*L>k#Mp7EiLok{_!^U++zfeKXG5C>s`h3 zFu_I13Y>7t-q?-lf&cN(_uc+Z4t3v>;~yy?O>~{yy~IA*J-q~-odRN-_l66%d(JvP zBM|YfXCMwZ@r>$&Qt-;lFXNGiAK^o*kS4fn_puvk-1pz(nrp7cDW{!=pMUxh2X8+L zd)@v#{#iBvm2}~<5t!8%2GdsY5H-590UNa9uyB#smf1O&qMi$dg@=35ik8loY5Zr3r3eNSCtP#p zbXJ;&#oslt6|YtLrc}X>wZU&r`hlydFjRy%!f95ZTIJXNYOjEJZdbOSDK9B}EI%-9 zI6hgxbgO3l;2^@rF7fKIvE?Q6aV8`qBLfdT`aTLN48Hi{HE7Sk>B{_ke)7}m3^hC9wawjM7P`kIhmOX2AK5o3_CEWaf$KlNLk46SZ zzbgyY@{{Kjq!Uly9}nJl76ZU(0ihzYAtAi>HMD_Do4X{kSbI|#1pXH10D1TQ@p%8u z8*$2+hoT=RVC>Dw6i4ka2#-8|HI6-MXPm>iGD>JAvW!{Q&#b;SDqr^#DZA#cdjVBB zMrpJp)LpFoRmG+F-KqOx1Sf3#)9o+Q^~v^&?#-?k3h-@t-A+DZe-{K_WtpVnzu;Wm zOrdy!8N4iL+)^HQ?$Zk$SVjlV)6$`H4mwlWYu%PC*&tJpysTpp-LtO0>EAf$ps{%G z!#i={VLRib(+|Oq-`~qYg%Ga)(~CB8;NGoH$)WXTxXsb>udCdADn)do>(;*uI&c!f zj*;_pZ_h5qo88W1qses5di%Zc_~#QJ;U)IXn{K-hdmlI!H{X5T?Bc(0)!27{&&XhDSCN5we=N%5gHvM|I z04AF>=*lav#)JtI_!-ZK6ise?fcbnr2Anbb6%YLO+i!5@S!dy?C!fOpJMM_ZXa5;b z-+wC3?@yLYv3Gu-~A9mPrEq;I4 ze8Aj(O<<3)gs7N!yHw$wPe$UaZ(4J5utL5R*9jFd4r$U&rc|Ubo@3lkt?=Ue198T& zlM!Tc4KFiY)m_}kKV8C)+kcVsss8Z?&Jdo>Sdaw9s?@DR-StiF;?8A?@qj-rZim0l z7=i0YOr&*(?kaJt9CgnhSr5B(RbgpyHg5ivZgiY?W>eEXZ))2jb=wmDT5mK(z($`> zJpFHObp5^m{vTE_Ym~j8yHNQ2m6P0^e(pawe^!N?OfF(oFagnms6T!0XPj~FL-@m$ zM{=HweX%CD$c@V9FbmQbUr)v1zq^}4=atk0YyhX)>OsNOTy$&%-6=G7QR>_pVVSv# zaRPL+bjOB?+larXRR%u)a01R?J^#ef_R~+^GwdG5U2I}RQ>Gq$@+0IiMkOjnAzVbv zf5Ybs$Cre62v^9-DONuU>DT`8Su0Ua? z9>#{`V;)oZ+OmE>O~LtR@5gq1h}Dah(bcTA3&3_eSN~(_YO9#AXLn}*`u8)JQ&D*Z zxBuw{%2{>edcp^|Yfp>UQFLcB3)nJF9=OAH1Dypcay^W+`61g#1*} zX!&~QjqhmDxCi&&{d;`y_D!guE808nPvETguQ}^SgoRUp2-vJdX`Gq^wPbvheuRpRgak01t!ixye~xuHZzKg{ zw#EKS=iB4zbomxr*A`tdm(e8P#Yugq;l*uc(6|t`BCLM{MK{CTNWEzud^)=j^Uiq+ z=TM<=PrdMy0Oy@|E?$25CA4eTF8q+Hb+{=bt5&VT@Zlry;rs8SP22xx?>YeED6aM= z-KlrklI33IUa*agu}x^E7t=cd5&{GWB!N&95`KCOy@p;AN+1COCYXSY3of|#-Yr{J z@6z4r|GrtRe3tIiWy$uAecHR-otZap+I#a_GVc1#HMn@`s~FZTjtgpV&sbzVxaj&Y zJu4LNb~_hQ)ERyCrI#^z@<}wn`x!kld`ki#&gi!KRq1>^$S`1nqLoG9;(;6R{xLsN z^?>#s92Yf{7Ksq9aaYY5k2hC#cMQ>MB4|JTRy>W$61r4=em*Wfb3Hs6BxnN=^p!d> z=l#;~I2?QF>G*Lj9aO|oTC$%BdYfT7VvHBxz5g|$%BVUZ#)R#tnIy?^rB>2Sa9><_ z-Qiq^sPfQ^Q~sLe`~SAb{keTN;^X6HQ&pxmV_?QSGl|&$*>}fd=1xYtr)@`Lh<`~o zq>dItm%|HA`wsnMvyH^6E@_89oPt5bNhkwtpZeV~_-cI`_lt^v_FDe=0)1DT7o0K` z!;(onQ8P%e*-32*<#<_N5`&|^JO69dd zV*XF-FpE0)vik_;##X1>KB;Q)lLE_DLi;;A z*3J8!QWFf(YivsV`b2m%uTiB-r|%-rchhyP0s`6utErbN)1uUekM4(bN+T55aLu|M zSTuhf+akLR4xTD{qQ1Vd>yyVTeXm)cd9FdplYEdMA4wOu9T+HY>9Q@@wTZ^oq)9>> zX;{7wOl0VyNz{wj&!mgnnvu!X@NUH84&uQ`2lmNq{V*_6`VdV9ZSJ&(F?;T6{OXs_ z8Z{`JarDn7luHVhlpK%IBYPX%Dn5(&ZB`tmCELJc7&?Kcbaq;WtBZ#pK!Wkxp^d*08W7BlIMB z{z;n{3A`T`E@0dlMn~TtHY+x~{#T2<@W0de?-GQ&$WM3Izvhgij5+b#m8Q(6_9V;(YgrvG8d}kOhMf_GpxaulGb~}R%niK2*onnRx z3Ey2LxRv~w5B)av5;k3zPy(@JE>#$nBo?I&MkqhFx|sE#?!R_@BL0LPDwdDWL_7{ zCjMDg2*;6VJAYi5wum`gzJa0a%r|AqSGf4%i*O?i&~s_j6u}rE z_CO3&tkko<`tmF6p)_R7u)%PRJp&u^$q2FBAj7+=80r}ETgQ$mNJ~pI>RtIXkm{5g zOC3Z~n_5_$bns)q9w31g!Gb9}dFhz~$ zD|$;R%u2+w@AW4%(wS?crC#u9Jq#&jwBjBp$fB1X64a8=1WNMI7D6w7f4-4~4H>4! zE~SCWZ0fxLsBKbFCYBCk@O-l|6|ZaAKev zsioT160o$VGDI*MVvqZuD^?z);tX-w5F*bIn;ftr3S}6rk9`6z0x?8tGBpRZrUnVg zx{~UaB%qcwpemD1Uy<*Gc|_!s4C7Z?{(E@a3R`&ZL#VkgGGX0c(Y-#%WBe< zOo+i%kj$#9zGNg6QSs=}BNfxA0!NmlzAT2}QLsa+|Dt#e@FAm7s}EVS+HH*O`P}Mr z>t3Kr^;~Lub=6~+;h$glHDgnZhw9;&P`$c$pr$t3l@3&NH305Ljs-J#=mIO6N0xhb zTY8Tvlz!-bSQw+9SJ`_6t6A?w(SUBlhV=|sv);VceeAJV=ix3#gW>wX(b$zFrGhTq z#Iaj;CN85B8<4sQ28eV2z{LQ^fYEfRV>K}QCK;66Jc{1+UVQ`Wz{N(pgxpqTnlN!U zCXe3?%j7=-nObeT%$0QISInP|!}z}<64Voz=) z)pdN_7h0y(E!nn~fvA0zqJ35R_=^3|*wbYZh4#nn?^R9n|L)(~VE6ZZhf8ifeXnu? z)$3nQL%*x8`s!0zcAX7rLqqGf%MJE^eRUbF5=tZb_wP#gwA&avgE33wDN8C$^&}gA zL9cZnMURxPKKN{$1+Al=GVJ_yfKpE@+t1@VNRIXca(X z?^(IP7%*d$r>u$8QI|N3qaa|zj5;BOY74Tcp2zp=U&;d(Jmk$jJ}K4f94)jl&||f|X`s_3)j~w)DhX zYq}e6b-~>D1}mek`z_hFy6SPBIT+aA7%(&wr3^Cee_CG4C=C*^@1*vJ0fzx;p+2!` zfdc7SbF2rju?$#~Hrz_;fez{DM(L98u`V+o#~pJlqmIT_OC)LpIcIpU?+HT9qgrq=eRN3aQX?EbiLU&QV@GhZz;Tp zsR-p<`&qI;Q&wd>9biMF6#O&4=f`5TQ+HX2*uXo1e9{43FPG^ zP<;Z!h}y#=aHMAgzWw}T3>r2ZC!KUMcJ9nDYFL5t+4pRW6F#aQN&PB0xdZJ%$bgs#X7-c7d7ZllHU>=Z@?~v(i0E6XYY*Nw?f?)@t#xE5zoBWi~S@O%=Xh+ zzDh2}?s7c-x9*IDY)L_aGqQx6^uwK3vdxiFg)wI&`!=|he+8cjO|RJW2G`{~=MO=l zH;N&be2kI8fOVXQl15kfv?a4fY^k8CEY4kl34N;(NL$*&0sd2 zVA7Z|hhyNtLFTrnyd18I5(c(n_tvx1L{n&Dq&IM%$VWjg!;CfP;ZU5d#fy4RGL~MTESsj1huwTu%Sg zMaamAKxsKoZS!0xBqOqZb1YVGNI-s}3psgV+~DNVfbFB6FK?uFilZJIK4b4Nwv=!{wT$4gJ*x3}GmH(q-U*Z%rC%$qk4 zv9YnWq#=@iWM^j~H8mBt-0}zGElcHOX6!&z{!WA@TAnHD+H7J%e30j^A~G#*x(u8+ zaROd^gI;zj$;41cz#~_@PCDo?Ajwxnxf?xF3i0dS>v_y>O@%-WHLgITEAi}-o+v8` zC!y3bu0R9T<=opB10F`w4vVY6@2;fnM-k68f=p(qS-E2?v2{yFJSE!?dYua1qx;se zKYtmCOh^UZ__zmtb?!!Vj%N_Ia;hc;U@nX?DFY?lef1iAIyH^VN|464QUkPdw?>VI z31QUt?>47m=%***?h&idD=JTCR`Adr`#bA<OJpU_inW9ePOdqcs@*wrN%Sul1l}K+M8Y>B96mo$%NLv+28=qlyd( zD8d$3QAx&Q%k~IHICW#g<_OGQ(2)#DG*+*R#FiZi@Rd*&;N^!WQK2%*<q{TTQ+Y-bX)>f&-(!*%J&c;`iSNRUKboO904)h z`Bv@PgY(WhjYg88_;SiTsy#SC2piFT2h^(yNLJzDydX9E%9w>niYS4%j3=`}H)hTS z4)9g^aYiksuJLq0A)=iFE9r-qwjbx7&*eD z66}&zwXdXQOST}JHb&r;e-6N(f3uVA;@}DZs+f{W25}xeECUyuv;!Z0+KugE5WfAy zh_vQ7x0sL$tjmtVRo{%|As7QilHu@{D?kdJ?Nag~_DB0lYoyj}TWi4E5@o8*VGJZ$ zuC?z_L_kX*@gOWi%tvJf)gT$GM-s;n(tiFlpPp+i+F$F=L5Sj@O>!Y@@Lz&HJ-{9q zwW?q@?iYoc{qN2HqJO<4~`t2 zi_V=38Mue9^3=a5C!B+ld9+Y*1WNP7I;E>;<>Gr=spfTM_ic!ATg*#6ISiKdjs|@Q zr4U2hdHCn_`B>%Zgu(rKW9+Dr7(U`KBqk*yF(DBtG!pOEFC8bJd z1mtQ45?t~2V>mZT_9K+=u+TvDn}Z-Ws-nCMOQQSXsB8e0?rfEZ9t6{f_5| z18n%$39iO~7z`J`csLg#sp|3XPe|P~pCl*tERFZ<)MJlF7DBbRE-_sS>-*XrxybR(M#sXT4 zdaVsYBk!njUaVXji}Oy~g=Ds^k_@oX?yFIfaG*80Z`TsM_IVFO1O$4oMz(r?8fiHt z)Ps|AaY65GblAaq2$p{m`gBX<{f$k1u(gnZ*2EOG(D+w5Aw2JLljy1B;gwEaT_li0 zDUENbcpv`lQTwh<-+)e0WkxkmW6VxAEIFQVyt%d)3Mv?QuLav`*V7hP(@yqpe0&8~ zeepHsE=k7LO^inqK@~-9gibnSF`%&_=A%UMgA7Nyka8p56zY z)Up^Lg?(@(ajCAja{PBncbsu~CEj^?89H|Gp)A`C9}P51OXadQ6lL`2L#-ovnhIrL z%Mctkq!1$p7oxCm4^_1yuyto7<}8TAj|)2>pSB|v)IqK!0|Ro$I)&8cdfc17XUkA~ub+rOuzL#t=)WNbllLojM& zCayetJ6?OICqnm=6eJKlwIOWY9ql1v^37ajQNiu7npaQF-20<#HS28a-<9ORx{oM9 z|89ktNrwoAF==a{och}o16p4s`8e_TU3mMgd3f`wUbyOa#X~ncD$Ol#B2OR^V-m#bv52k$C6BAB}!{;B)L;t?zDBi`0o%9T&LFXbB z6>5&WC|ct{D=l}Sn2bp|uU%5TNJ}lm;Phf#a7HH9(4cd{vP5KLMqus67!>BvN>D@_ z2?Dc-@oV*%)aYdSkF6Po^SWeVNSud+pSuqAmVL*QK=OGQmmww^_jS+0+tZ)K2g~N6 z>)7Klv{z@uN6?s_5}Ja%TuubqZ!k)FWK;~MuiAonpZ^2D%KaK+yHoXw5`0Afh~Q~j z2sb#LBpPm>_r!dJXyWk{(oIRnSkG3x`SCxHzaz>lI5be5=(MB z;>|U*E1=$bOA|;M?}RiM2nWP{3+HwZv<%2v!$L@;|LF$C!l3O(AQdImq?j3@ag~@i zzbBr5vnTJA(NJBIPd#nFiOUg_7tg=e7Z;wj6~nu%hmQvKfmF=Q8cQ79$>`|se$fSG z9Jc|vl@@NUP}s7)8XUm=-Cg^$ZNXc;pP#7F#nNzS>($yCSNprq_cOMGR;M}D(rvG= z%wW%D^C7jX?`NjqnZNYItDp8ZzqR-2@UD_$W%TeIT>YyJc>di!$Sq_9b4e)MKyntW z4M7oo06aJ-84%%J8@QVy*EP$Mar_zM@Y&zzVdUY(DBK!h7>=4di^R=hOhinU@l?2& zYEr0pY4vsrCrL9ESnZhDpgFzX=auiJILLi zpH_iOdxugT;yXOBW-LNz`{6UQvZ2t{)sln%nWS49BR)AC;P)0^SsQ{v(^XadrHi~< zQKch-(ygO z2AXO$t(jW4^wxyD{<06|w$<5sE;sfp6@h6Y(sN(b$dekX4(ddF)|W&p7>705Q~A2q zN(Be4OQlQi*{ZHOB+@=#Up`4f1&v@*JRreo&Nz8rG>%OOs~o{(!>+RRJ=j5FAXv0! zcQXC&BxZQF}V5OF*o{=nzacN)@4(z(sHuIhJ3jRPzcNMhM9#7SHV`kH@0rJ`XulA^|7VLrh zGf@3lBq9Uf|9*1B{OhGs0d}XO>Zv zP(ntdjM9+~i5?`xdC{xuE-YVNgw5M&z`i{ixw$-bXVwAY+`WX}%In3dzB- zJ*i3-9b-Jo7-M1r(UW@qUHN%wYa}KnRGpD{;-Ey9!8l;iBt(}TKIDUD_TDJw^o}v&F6VnuB$oJF$-rMsxD?wax`Oc2m`nAs?Y>HdDyX3C8rMwT z^_IDKaH8jaeG$&*;e1yFRaG)&@?E-%Gi6ryLnz%<`?N?y=M(bcXkE-|}p1nHY(}8ZV^@_QaREA+v z*@vdR)oruj6s~l*;tz}R$)>L&pISET3pK=3ytkiNDcjBaRjIlT)Bh@0jnVkn+n+9u zfk0!@E~724g7~0L;v!uf{Q3Ejxcd6T@%eu@t}jFj`*r+W!{cPqh;-No3jB^v8C#h{#nwv_fFqU*K5Jiehbh9&L587UlSTq`s@ zEbg?f8uvaEli{O(|B%pd3`~o_KuISma;$>%nj(FMagy=m!B@`7$B9;**tT=n&hkBu4D`muFREU_s#S*kDNb2Fkt$HQje>KY{Y{j zmLV;M?s~LQn0^nEsWon!`q+=^N`1GJzH2R~>WUs|0X55MJ&UH^mFML!x^(?$b2|RC za3B}kaC$P)b~r6WM9i$gSh$_V!5KuIBCCQv*V% z!<&l25cKbvi_6d6h8JG#%aclGG7D|MEX}^@Kb@LgSUW4=FkKc#002M$Nkl7x7C1^bglB z!Q1`z! z2rHJ+{UzxXy=$*4K6=2r=z@)iCBgjqN4@akgR?MvK%SAf7ZKKbUKoP8^HSN|5F9af z7w*4d8C9}rAIi_lb#b`s*9NDiVXeU( z>pAzdT)hL6CBU*ZIx}tCwr$(Cr)^K$Hg~VsBx#?gu+vH$aF_NKLW? z6`7kezJ|E@0Mq)MKUki70X=YolWSQgL~8Jm;ce93=Y*%(-#XDN+m~_(K6Ry}F^d&C zNS>~1F=GN70Ybc^9Z!L9noV4>(>~s*`~qNPy3bSTI0n6IN9yyj*TDzP2hY5m5wT?V zB%whM2T24AH;9Hx`r#3c>k_Irv%FkounaqVIBvW=s5)*E2TSi}xe5T{!D08C%S_eH zw^hgm{)LpmDA+>@g6xQn8ttLZBS;byUGa3gt!B;5+et=~27Qs#6-Mo!2DTGA5PTrS zhw0Ntx$rRs$2oqF{qt&$x{%ApcTifZiD&_Qtcvy)$E6D*TxkR2uDbtj{JBY1EdhrJcSyFl9(EC0a+O{5h5}#Uj+bDdIV%@ zJ2>M9C)hxi_rm2vFu#{XQ5)pUK&(g^sb^&NDHCvJvovO(NC44<5xJqHd$VUBAQ$Ds zb$k69K4Vo|)HtmNcT4V<2UpQj1F}X2>G5af%acXr^)Ag0jAY-FFb=>%yw?3Erb0N(Nfl&J*X}9r!no3`TfnuY zinf25eu>fjo95LwKCaUuePzckO{ULf{L4y@mpTg(IYAm=L4GsTEhkfEV;b7~zDLzY zGu|Jx94Xsf6`a1?g67L%$AS0D)6ckCgAW7px{$$SnJkbo$LGy8)9;Nvn%5e6d6uR$GpuyRXK^1>IMm=v zLZyy}0$HD_I~Eux3|TrknZrMFcuu~1&(MKQtoOp3V(OBn6Y2L^g7D1A%;GDFKD9?8 z62D!5sG;A6)WXk)OIEMNQihC?*$#)1*$jsgJ1V>qOZH;sNl~E9Y9pX>2 z*h9?tIB=Re)l8?GX^|R|NMUMl*+_-yxKB!}wG?oZC}*UXe+2$KHNwYHJy;{Z9nQ>* z&tm2#iUjo<@OghbtaCp{R|I1YCQNH7ryh?-aTr41)&^@)FfJR8?V zP8L^Ht1THgF)zkPaUP6AX*M&b z)k;zq@X|kA{SLnOZG#ho#c>?mTT^`m5KpE1b21xoE$Sl8hQ7|`5Ap(b-cgKqs>FuZ zAwO=6p|i-9XbPN=d|l?mb^lDG>pDGKbbd?OGXvhGeLoECDz3vo|G9J|XQF(bYCU_U z+_32i#cqNV$KLuxbAkTM9~3E&!d%i zm-3l7uQ{ksUG}v>MGK>F#YpNi*m`}x1v}Ugr9%A=vA3zpen64pyJFlDhMw2J8>Xkr z{yu7PM6b*>TSTfHZSK_9ciKR1UM|<6cnk5`bpPrOfki+MgrO$@0E!<>T-45}otb=3 zEi-Pec6?v^-?l{s8Q>5QZ;)lqg@>-k}~WfZF-##n7Tw1ZjCLzr(o^@6#l*Q~uY- zIY_CnD6#%Qj4=-d3$Bt+SL&BGH2wD!r}D1y-oD7y%EH&JfcIBR+HkAhNToMHFf`uIWx-|tQ_M_W1?aL&H!(|_vs#EIjB7! zH^K-(9ZZhjT$ShVqxr8qQKorS`#4gU=Xx7U#T&5v@mbu#j6!(H-LL+Ez;24H{!isI zQR!s&`>{0l`@UC)q4%4Udk}|r9}Q=cNf3?%8!? z__a|B-sfF~R$x0^>%sHD?MlnPp*;!2G_Q<~s^t70*4W(?D1W~i60r{GknNt3j!=Qg z^Ot$8`~6d`H*ONU1Afm+%q!x5o=07I_Mx0yl}rBJu!HlzJ;ChNrmK1CIfz1##@xDUlC6^4AB|G z+2Us_Mu&r*Du?YxZk(`&&Y!h8hu>dRfRvi4NEeqRAuke}m=5msVLch|+x#Bh43Odv&n@D=wyz>E^b^e!#2;WGfl0_g^3k7c6XMf zAJMnd5?ZA#S2$RdiBL9)L}=k<4xE%|KPBK;N_uo;s@cAEj3#xXA2RmS4(t(F!Rztb zm1I_>zmH{W9(4~QKq0&65F~WQM8hSohf6?TK@b+gqn_*<(L>f=4AFOwY`MF?l(%5 zfW5P^dNxi56`$6Sn4MN0PDSZTTUvI7tBOQzFv{f1v$>r}_@zjxmdheoMu13dY6|4f z4duSR_=r1Fz(Q*5vEhgoK*NH*SspmY@V}3q`+)%l2!}KgEtUqE@mzk-= zKU4(br0z?%5>v-t#`IS?Nf=HZkL=zz>7AsnG4ESbjY-HAV&*sqXa3+$Mcz;m4-}y& zlC#oQD`vJ6XML?7UI}3B)ox>eqvpKs>}EVDs^vUmzvRlNKu8wrpC4xNz#(ylTOGUvZrwQC)Y$w0(1RO#jCB`H0gx2vewffvRn8p1UKv7;I;|WeK5IpIJ#caqtfD zUz!aqCig&nM(K7E+{3xy!;X-w==Ov{B+p_u!Q=|MjnmUaYs(t`AjczeL`-E7-9Z_*2omujP^>B2bY;}FfDJP z99yShNxeto?#Eh`LoA%0X(c`QsHZxP3^(pvYFp%_^>ifT$79x)2{5vB`>G7z-00Wl zEtqCIJD7df5)Ox6GrUsc7UDsk$uc>5Lwxy%8Op9cUgmkhT}&|uLM?;I?x6A&2Y`=7 z)82u0ZdU)g4p(Yt=)zQ~|8esCl+(t9p3Uh_Q7Z9gVh-23NWH1><EhQ=(_4U;rqV5qkIe^d?XJ_w!$H*4c_8!j$o0Qj#A`WCBGS zN&+eHF9M#J%fNDc(yhDiTd+uQZ-z6F4beqtez`4Wn|$~7{AFa4VLRL#VQ^^-df zp`j-Zs3k!~QjM`zC{eG993>2~?xH^cVy%rtrDR`51#UfzFsPyJcTq`M|As8uR%vk^ z+{^@xmMeCoMeyslBrr%R{c@_?AEm@_y#v*XHeKKdEEZh*;~{FHHf#A$8zB~RohVFR zRWdg&u6~31BNDBBZhY&obh~eox zUa*?o?ub<8x9P%`(cfS%haqL*dzq}972~uY)UA%!^dX^U(I(yAS3rsAX?En4(|b#6 zA#6d`FhBk>daCaEJGh8d>A?3x66Txf_tT}|y6xn-hIxKtz0aZgKkn=JQkEg=k3PfX zI5bUVh{zz~caeAlX?hOK2>PZJ%5hm*sSNf+=bsUNSn(X> zb%(nV2r-}`(f*jo0o8@g_A}9{x;ET%>-WIJYQUagvF;wL1^E%Odl({ayDM8x(Gp!=c_3b zW0RIIBw2mTh7&%K&k-K3Xk@dylZ?ej-Od`##)=!`Jop(`EcPP_yG89>%l7f$EI4|+ zX`>8jA%`>%k$NTk!0aF{gQbKb^uhx^q!>v$f7;X$RwXGQ8_{~)ftPyw!M-NE$;CzE+{7Lrh}m5bU7iODZf=w8FvS@(DPm@Nt^G0q4(n>(>nH*Cc%y+Y2On zBHU2)sEHXR|Go22-xSJ!3Q<3%2*DcnFGaF{);U%P<&yR+nilP)4xt2iL?tkJE8@rE zr-;me78xYLAv>ZjH$Vi*QFe5$YjkXIMUUa8_8k>G$JD#qFq}u~g?MNpH$+53xDZlB z$?XQF^oyLDWRDKT=O&?&MrJsm_C@ak5r>nNkdZk2bmoz*i5E~9BG>-v6-8+3;|77X zm8}k2E42-JWWr72Nb9dIpH?mb*K4`m0u2SbEF^{SU3A%eKskOe(4W)$f zhmsf}inS71KICiMcNB05bH^Xn!fZu@Pl@IdQnb}ChOlO~d9fTz1nH3qcz=Qu(x?hs z5_9xiShxL*;N`Pxqk>^aG~%WzKMnHG2U-vd0SL}r|GY%e19XY%R!SCEj(Vs;L+^S*V;C8zNaT)rks6Bd_BAHqvv z6WEl|GwVAz2`0*{)TGBoCkeRso5NyBCHfOlvU*o^TO0pc!Y9VgHqf1q zP2Sda>9hz>>deno`h6X!m#g1-uW(f%W}!&e-zR7Aql4bUg%huIvYzjUrW^l-hhi6TIx( zp9{-t3&8!Fq2R@KdArrkXhS6b9p>%AFZubS2uyxPf-u0S2%y?D90*mUJCm#&z11n7 zd>K;g!ikZ`3SP4v@{>tSa(_QZh$pV9h)7e;NXdAdn=E%V#a;v9UlCnqw`)KrO0X#2 z^J21C0_v<#dml+!+F2XX2n>1MYhQ86;&`4Cwsc0;*(^3$rW-Hlc>c}EJ&cz*D1o|6 z;8}lO^Ty$54oaKjk`;4_W& ziPT3YE@f{-p*z&K_1U!AMC_c5F53B<^zqDWdw~T3S9h)Vb zvO#?(E{6R35ok0)S`ZNk%t?{#yo!r~u?#Qc&dNIVuc}P&+&@|TUP8sWn)j6oSq}9E$)AQpbJ48V0&Z*%yCYVrrZsGf_kHu(8pw zXyz;#&L{GHS^81`ekJJSUM8xqh{uTw<0X`qhPg|Pn&BM$aYMv^+Dmhs zYs+0452tsV7P#VL2ZhG22i+x1&Rf1%;1S|4?c7+(+ZT3_RdhrbCC`W-FG+AIi~ zT@7pYe<&#%vAdS@e7ea~RIa0jWnhsUEHHS*L**;+AfUkZ$^Tb;~&r|DPuJ0kafGX@H%7QBdg>w1k0!Z9$8HN*@_)7t;&{`n%GERi<& z$rlq62c?P|_m-f?0X%5Jd{Ms96^lHyeq*D5^Mf7@3#six%OwZ2)b~b? z4$|=B@DHLnFkUMjGt1tJrSA8h2t^YlIBbC-J0gOatpc#bI4WtQPvBU~)q3Ej2!6#Bqc9b0-qI}p+Vt%8Fu6u8bvx*Y|1Qu+|0C6K}d481cSrI)LvTC^d1R z5vxh=``ks~+)!>ENh=$bZIy~tnphykj1Z1>_B=mo+g~>69|cfD&Er4Ry9^|s-z-*~ zuC}GYQgj*CPe^4_(-etVCR}JzTbSv_&HCIF38nHsdToAY9J%kF#@{$7#Jf?g@g$)Q z?cNNy?FMZ8%_p3&B_*YvA^l6WRMnpOS_gSvB6r?VpsAd4P1| z6_}^pR>RMhodf%)?|HP_lQ_lLd(4@`x#N?$@=Q+ScM(b~lndx(i#W-RH0ko`iBMqB zxARCBM`dq^!p!i@G+-#w1(wxvIE|7ye(QJVN3YETu3oeZfG04Vv|xfsU1&*ffxX97 zXZGUL8c1>eyPNlC!`H@71%Y}*7)%+HeDD;^H2W|*m$j3vPi zE_h+K@2ml$Ap#sbe5_o?0ahvbPbpZ*#S(KQK3P>lx{z{O+_-uos(-{6&4fl_d?*Xa z({-^F`;LEtplxG^g!z)92^4v7VTQVJB*bX6bQ(a^pywvb;kXGo5?wqlsr=SfYP4Gu z;rT?mSFc*)jxeq=4unhry4asE2Nz;?o0mju6LnxIgzEz3YI2qx!!^$JB9>l%aw+!o zyh_&>6UMZ?IJ@`PM_#PPn%UZE;jIP)ukkix=<};#&v>%1Lt8Op(izIJeYD#Z&I1ko z_)zedh83ChzD^BnDz^BUs-T~medUO!TZ03hw$mb#zdsSOy*+*aYotl}g&BQC(d~MC z!HiC@T{po-h=Sq^kEm4Up}|NeOMFHwQ{;L%2)_IWEO;Eh?}KjG$Z*q35>Kd~0I;zs z^IfnKIZm*y%*^co*vd}UF|jK|@Eot*IaAK~0I;1Ga{`;OaM54Kk3$sp!ybt(|LVt~ zdl(q@!X-&QF$F_|!FMQ(btrmCCaTgO{ir8&99_8SD4vg;mY`zS{Lh1zt$Ur1?)yhy z?yn2{qxmM1>^3W3-bna)6T2e>mjF0e3rY$E?KR)j_)v3#^RV~tcGv;%!~N5i`MNmw z3wi95HyDh)Kf@K2W{l<-3P-3MKFe^BfK z^|J6Oax>Z`GnWD}TY&}VL%oH`pe1!+U(Dmz0DIa6bK4OnK?r~sJ?w4?ruI7CVTEVe z{>G0g@s23@eG0e+mS7|3e|n+s59ibj=R$?FUab@`I*FI}HM!IZo0BKS`Cr1DU>H#! z5=d!#7jiIk-a<~k-FQuYeCTvONb4WS5~hUh^NmS<#k!Pc9{Am|$i5q-J0|3R_{ud# zSvOP+__O|a#13~&NR|C*@u!~TR>chU@tHbK7yy0R6$uA|JO+i<8uYvkhvY_-5E3n+ zIp_+jFglR@aO$ls!O)cyY|O2o&#v@og?o@N3kWF^8o>Qk5WTj~sU_}Uh>Lw72JvTe z&xrqGzpX9FiHMWj63b3sE!fZAG)`#xkZ@tn?yr+AG58_{#QpG=Xd^3S1~9pY-!X_8 z&xipFH~z;T<#S^}<*6cGYbZpUR3Vw?Sdjc=yF_|opf4|%dciyk+4l2`0Ik9Hk5~Ekk z4Z5sE_fKy*6Uboql{#6ADv_PH^Nc9RFyi-o4smzBe&DJPL-2CyH*v#bFg^_n2oQuj z9yG*vEO5)UKt3s=Ffw*a-%)J9$q45Qzvf}j;;)ND+^od-M0l4;DUqefi$fpEMJ@(~ zUrlDsVgC7$a=!(@ILxK#ILi@U(6GD{e3g?t6n^S~-rp>P3vtP<4Ss*(qM)gYb?hCV zJ5252+&sbXISk1iAkCayas2Lv*Aq`>UNzwUn;`XwG(*p9?=T{QA$tQX zbIBC1+)BS&$j~b^l$PRuUzh}ug%+7HQ_Rd)l@tIB2VCG+g}vgpEyW({(d-}$i6I94s8YllXrgSDquKS}N zywr8j4u@R(Nw`%FaWRpG%T!=Nb>M-U2MYxL{EjuoM{x>%^!pSsAAk&A$S$;B5w2Vl zM#!;AF=Tw1A1h88TIF2y7ocleZM4MbvU_crS)6e5s{iCN;k&~s9y20#e-%{<7hN+1u@>Of7Gr4%X-<`{yaVu*X{fz zyUJ4UF4#?&d>v|V_pkWeQgUqEZ#iNJSHwlP*K4d#nErJ>Yb40YSwZFO<&9O{1|9I! zW##?0HG5K>_S9BFfbT5-JA73WgA7NCS51-?WyeTx^TuIrf__F&`cQpylZ8OTCA*{kBq zgwho5be{$M4jqwy{(}7wUAEB5 zOOAs2g96BPP~bnA=_iYP&i}5`O3V*Jw&2@G0!+T9GE2OB4MvC|>Q_OT@ucUj44hjS zy1_O*1Zy%^;S27W{x|DGz#0cgC)r=Tbi3$w@NOb5M{E;{M4z8@+QERuK>iD5 zhI@Lo>cpeB&gf{f#t(rCT+}nRp1`Pu#fsbofva|algU{*cSusV13Vf$p8iQ6?vG3A z;P3Xcmgo*v_XoaAaV<1lT`4HA@R&^z60%cXaYAP9Ay5kH17gMLnX7y{O!OYS83f9{ z7gF@zPPEKi!{DPVTclP;7uf1+?brgUz0qTzzTTGT!+plHDEM;ZA*;G_<#*ILp~OOp zJJav+X2i`#n;#&yk0iSCi}$&ZPWYUd4vf41u904%pr1(JW;5ndMMSxL-FanH9%?Cf zYh(FDmHNbow%P-}^x{%ft9^ONBl%Jnyk8nlCzyqDO@>gvWA)4~dC}zxj|0x;-@QpV zKh&(WtlV-@At|UAR{OziG2W9xmSP&Z95x=Y#T8kzd`Nni7;_3yOf~HXYyd$9(HW6S zJT~catR{-X1?AI%tq%|qoTY^;6AS|Bp#>vXdCmu#wD~%w1H4$e^~l@>9`|Eht^0R$ zn4X&vn%$&htzmYPn9ns0Y~2LNu;Hh)XHQwqAMBg&VaZ8~FPKq;(fiEu@UDNs6Tkua z)_)g#$~u21-G4IwSwrJZrX-QjlLi;QIzcs5*_Vyrd+bUL%1t#*PJpQ<9Madum(LRW zA}>(U2C}`PJ`;9W(2x(?4^)VZp8(?%Md-cP3XFV#z;=hTeNN{R?In~?>1a0^okwXV z{?NGF6Z|41d*ORkCW%J`h~%1x*0BA=`;#GSp=CkTB#JaJp_$CX|95}m!v4n+4*yIs z+gPY#OiXjrLIFI;Z)n-O|8^cW*LPz=x`cRS=)GL26o%lIeBB6z|dOT^{F3jeH>BG1H6nq5f< zLQFo$h-boIC`|?wdBSZs=SNNUpp-*lAt^2j{HMQz_7FY<*^yvZT3Oi-i;LBPXAes! zf)NV5x#5%2K?Q3J9--%$L$pQu1Q2AYM&3rS@*ew$u>y!zc%!W}vc^(JK~li8Fe5|B zyJ*D8|IYVRLNnA;H^df8yYv5G@c*7NZ5WU{pqf1QV@}RP z|K?D{iJXL_U#B-2eMTo5$!XQa?K8ygid*b(ehAg@t1w|k@p52Q-Vr`Ba-5ZkBrDlx zPbqUIza>RAryoVkFw7Wj>j@DdY{*cO=Pco*OlW>mbO_*)RWgqpKLf z={G~iT@HJX1veEN5PRkq8llLNC@^z=pqwUBlh!KcGW^uB^ z{A)?bv+iOR6;tx8lQ`2d!L;z5)5K`bTdW(Wo$RBE@ZhXL;iHkfQyRJ|lVuW#f)S~0 zB@8_pA&S2h3>AAn(ZH1~<_LoP9dlR!)-C}HNKKU*GkxLbfNpux0=Tt24Z&dof1Kw2}?j@S?e{t)C6k$UCt6JDE z33hi+%;=>~@i8jM;215-Uu%!gJt~7%_jkKubk$K(m>5BgtVrDkaiYx*GIBziF-|CS z-U6b~oge@UdK-?&O<3HN^}st=Qkht-s+ZwCXyfHNm?%K}I{pE!D+G3Nc2Iaw0*$06 z8oy4MFBY}%785RAL{@?_#+#a}w3F_bh@~hux)ToUROLg z;(8hBtQ;BKBN^Hyi8KEI79fHF0uQ2Ga4kmQAyt_{ibjWh`{HePc-+q9#xJsQ-PM9S4j}!iHg}AK3N;oCP96r- zv_>0?BQXrj8Vhn1K3_&}rFm$!JEM?Xqo=-yN7m{JM2*{yNIgW?)FvVRL6i_jHft}l z14S`}d{q|5ctRw&f|SNG)mY-+utBl!RxT&=9AlLhVWaNU`ZQU;zMUtDa3~!JbqE4+ z@$ec7GGK8~iCNUp6fSLwMY@s{b6Cc-_x@gMtul#Iw-0fcKpo=}4Al+x@sbPdsPF6# zO!L8z#%<(qe-4-nu14c;I^%d6pP~*la^KLb>vD|*N%;?uAi%)?lN+;BU6dqQx*13c zv-5JhjO?pcWLUP3mLT1FbCs_W#3X~|>xd&F!r0pbW=S%r-?6BVee%rqE!JxCG4w;W z%j2Le4_6hx5{H!WjFPIIN6fk_JqTur?T4HvCa`V%*&?z9c&!f;ZWB9B^DZ z8p)H8+%@Q{9W} zCl-VG;NEHL?j}i{f9==1!2KQ-m58MK0wx7hL3~6V=ktEST$MG1_uH#(rB>Y#q6ZN! zi(Kx}xwUjdPE;lFhs)eWj5Pvl3uTMJ21mDGbUgIar8HH?Vzn2^TBbJ3+Nq; z^bG)Do)>;V#me2eh4d(#uRb(h4_)YVp}_A0`^94VQ=MKD5SJP*-qf>(r!4U^ROt}& ze1@Of{7&=XB)h+DBm6P*J&^Re$n;3F$&-iQAm`A;DYj*yP78uMQvx3R-v_|}le2N( zztP|rGf+&lE1^2UTd`{lZ?7l|4C5gZnwQ@h$xdnqPTvD_3G*VQY9+ZRScp3|69b^a zt-hNVFyvj6tapGMDw8r~KpQh>j~@rv`Ci2E)8h-IHjP%O$KYKcJysZCGDrmk8Wkd3 z$MkIa{G=znJhrKor)n52{QEakbt};0<9&eOCXl;d>eFG%gbE5a1jCNXZgF8J0s-7IW z{0sSaqf@K4NRw9`fm-PSp;pS2QH2&u2%XArX#^jFLd?hm}3=CBTql z14V+(uoM(r(C&#c0LbgPVi5Ah%mdJqD=vuacsrnNJCmP}FbixQII!hoCqE}J{U@R zpr++N{zyQEt*oR}Z%56JO)5f+PG2?CZH9hMV22$~7zG8HnJh=)^(1E==;`k zcS*wxVP9w@W+lrg6>tX)(6VRM_!O{`nF4$}`K+uLE*5x6wC9%W5x;Lr{l|wmF)id^ z8*{d5*mT+m#+S_u*g@@J*KtXE7#y=wVOS;HCEpI z(=&`ZZIEiH+{X+mA0*B6s(lXNIw~S1{#!C~tjH!QWs7c1cqD^lXVf{(Z44_J!X7Tk z9_*}zwKN?oJcCbF47-KGWOs1i*#HoA4kKMh4==Oy;w{|*heuvV8N~%)E-x@nVYh?^ zpbJeq&%L6|=IKy(;x0Tg<$3PN=t{4BIf&1E1G#5fv5I7rrYJ_UeV zf0Az|P~wW|Qd;8EKwX7HX0xtEb+vVTf0uf)*l@bG*lZa8fTS=c<+Sz%b(&I`a%*I# zvdqQ?Rv5Bu9y)&gH?+;A!gRolD>kUg=H0|;PD z?W9DzUwp25?j9M^;14N~e?gh%{Wk1;8&*9$M8)o(iljMU0zW%&|0O$Jw1t?@j!)}S z;8P}?foP5B(;s1gQ<=fl7kHVd8Rc1TZPigBHmF9NUb02poNUvtJu~UF&EvU`c=k1d z`Euz6Ji_*SnB0v4Up|&?#>D>CxTrTSb4|&VO~#5%YahR#*;wD96CVxxd8#L?_EkM# z+xq@+CR(QGC*60_RD<6uhcDeN&uJG-rIH|k)bE;L&PtZz_+Fw`k4M%lXS_l@{M=u} zF7;W-_ruzqkQq;b_nzzPc23sBxG0pHd(^7)tkqa1RTO|^)pLe3=L z>xZ7doDTEto`88l67m+}lawCy-zI1myZ<};c+6VLS#h&JziV(n%#3rrw#YoP;tsSz zVueWI&31}V4+9&SJG)tMXY8T|K|b3IMzSols`IM`!mfQaeE#v>O9GB7?7giQQlBqW(hWH-<@WjoS z;H>}Jmo0>EOTs`$GCIyn;=VT!6?0~|KQriv<`UUi@2-4jeSU78;V`^CU73`llbA9= z7!A@|#$kq!l@P=-2;>+uX}kQO(j5(29>S0~J$V*v}D&0F|~0b{aUjEfyc0?D$M~6fvymHnWr4hq1}j*a88Tqo9@-|sO)|j1Fb_xSVRDHpAzAsz zO{H$pQZ)*DOS!48RD>{s@Lmb72!@QnvMi!_h~4q)&RRe5_2q9lMO~A8%kgI|EJc=w zz`t>Vqz6zZk1ahSPglI_h)cGMKDD`xV#OLVBtatjKIYFkez!*WgOr7H*$0Ox@B`^& zI9w_hTnFF9k6Y9vi;@DY;x_p;zzl7-b?bV}`1RSf=*2bGN#DEEnL5uQ_22}{h9ccB zu7~QdX1T5R=ib{9SQbEF@_A0pmXG}mo|uVc=NdfGT|LY*{ZU}XRZxrVgm9hX&nh>% z3t+<7=ysMhW{j2hc^)Z`FLL>r`839VlI{O~yuZMo#s_rrZd^4;S5_fC%JN0?=S7$U zqnwKh;gWuxxE1ARB;Gt@)L7%CIJU|#34I8tS|*s&v~4bp@K&@_r{+x@6;V`L$W9$v z8sPK(VHDe-lRXgV}xuH?JE)Unxxs=q<&}CJpT@T;+0-|j?9dE3X0+hob z|8)eEZ0W%nn$0Av&!%yWZePZg_$+ma z5L4H0HT9dU@||wwZHB#f=dP$o%E`&RaySm7ZqwAra*@(AC5l!Pm;}j==s6R3;E@dF z(w}(*Da}FLs!=CHF(pd*3q?cBd<8dWK|@PiEE-SKdbeV$(oBX&!(N-W%|kb{L!;yA z%<>gF<+EBOA6P@6TF=Vr;!rTR)I;zz`T zHh6npQ_;^a#I)|>yf@NvYdDi1(o-%@qlN^0?>*lBJ}qew6ZDXOTR7SPK=({^nni7^}(+O3veTG@?axOytECV`wcW4m7ny$>a~T$|P-Gw>^_ihKLYPe; z;Fa0})R;Rl4k#f&Fczln9_%q&h*ceBpW0W|cW{8rNk=D=x}zL(tMG&^!tsQGc}WSe(9+D2V-AvXG|c@$ zv32-^!?R)(HBfbE_xRl;xbw_SnbZJQ`GTf4Nm)dLL1XMibs#AV`n*tTjJZK3J=PbEK~L*WVYe zjq`QCq@Y?O!(1o0x7#c>q zKaI}I_dSj@&Te@Yz@Cuiq1@}Z&wjpL4NA#lxmA(}3YYT<@K`Zo#k~Saw|eNrA8Ye* zF)}%}OZBy08dDj?{tDc_AcX^>%9Ce$V2?Zgmn_2F#N>3wM7(-UFtSiUBQZ)vShJbc zNj8*~bdba*b}5oZJ|Z)^O&l{2oQD;^ z?EmI_U~(q*rhVLZv8rC!Vr;=N&vxZZiyOE(*lcpY-@2wk`~TYe&cCLXu5GF`DJn%e zibxX>LAn$Xr6XNxkVu!_dr^8-Y0`U143Q220!Rnxga}9|p@k|^LJ9EX+|T`-%MjizC5H1Q5ppl(Mydhj#HS*^e-FYP zFStH0Ru9T|vC}y%;Fro`K=RjMTMqtq@8k^D{zinG5&p%szRq<>l_Z zJjZgzyx<5RcipD!GnN&6c|(2l6;I(S?Us*;sX;3#10U}8@dn;@&N$pY&AojyHzC%b zBkL1Av1n?-L^QLo<|swyz8Z&ldl3_dB~M1RxME3OM(u?pS=aLv+sqhBSWD|hT=}mZ ztOgii;B_WsdoLm2*9)!N?~#_c$K0h?ocN<+I5gpld@@f{oOSv8KIGgHr={Wb9kT9_ zKV7KtrseQk0e;VE8&qw1>gCw!b!6B_{9$|>pj`u6DW?1wU;Rv5!KSXc_|iB_)QRVb z9psx-&xq8R9-<>5DNd0CttR*Tgtv_JBaY(5JVyPjM2)*6CL#(mt$a(iMBuouqU$UZ z>m0S36o~4&u1>sDF(*-MXX)|Z@ZztxZIR*1F@zLK_o&CH53CNWACz@XN2Txzc4eM; zw{MHxOZ=fxA($*@9r;p5FRbauGjqRzPffo!8y|WcHmj;{I!de!s#nunTYpJBTRs@= zyGT|@Y~MM#I^HEQ9|6J9C!xFP**_nsO3)L2Q0bs#pi4C^sdE%erc$*q#TuCj)tj2V zMF{X&V}O8|=>4 zMycqrRWjm~JRu3Dni(|`@8;!1N%l(C_cSu%EX_|-8TIuS4XEWk6kZe1U1=G2JYm4p z^a!F;^+yx;LiUbJmesS)zn@e3Hxk_dP%JoUX4~q?$W*oAzT-Cq=1A#^@flV7bY^PwH zYt-1FzQc zvU0JPJYZ~oBRN@~4=&xv?egmJgDOiDeTJ}eQuMS$cO#7jldJbubnknE7E5Q!K+s-I zec)a#W=+^OsW>Z37TwzeT(0~4qG#gqi|#tRq_`Kxd&L1yy#bNBW`SaPL-cMb;}wx4 zvh@+a`sDX>rA|jj;<8=*x$=S>n@5;P-+^M9M_bLF1s$Ba;i{v%g^)F6LV`nUOGE&|drLdg5aI*iy^@CjMZb z=Gopk=lx5$YEL2#^KH`3DLF=8R8VN^>dtyM1#hVQ<=UP|Otx#FA~05Fg5foQ!G~a3 zSOx+Saz{z;#Y6Lsn6f;E*Ke48CIQV?u%P!}F!fj{1fEEn1YS@?tIRdIJD96NljfT) z?sw<6&CIhtSt~pfYS|!be9<@fc$0FbWO`!y&=jJbf2okV3j(IGpwlaBmn+LN%B{LA zDxW8^afEu*z+P8sl!qey_GNY8UmJ~9kNtNQdxX?JmH#Xb1_?#X;YuY^75fGM)A0v4 zf7JBiT2WO&v6J;7^+9#id*gnuyo*>!lCo?MQO0fVpGGd+i zxtq7{b8Hx=_QnRyT^cz{V`9-svhr~E%g~->?O{Jen=ndV7pN&-w$kYXogAm9t&{A# z4k7i(cfmTnJo{3gC^`ql>@RsIvdB(UsE{1xM5K{=3!OR%XXI@=)!5&m za!smd!7h1r8c)U(ZrFvK?d)%&X|(36^?dVi4_XWO4Re@Ut1{t27}~k%XmMH)Cp8ojs&>2lm{}z%ii#6<2o0HaDt0*RpF)foJD`eZpsn5xWAH}SsWZ{Q zk;yaduWMRFOEmZeM^RDi>Gb)U>3bN5w-d~Qj7f;))$R;rg8nkgK^+PC6t=~TvHftE z>I6Vso%c>F;$z+_!zOPeO`_hAoKo)r_X#O?<74s+iHU)mACAYx^zN zCl?FA5pJr<)0N_tT^SCMUpn~#*-n-9>twv6?OXch9HG3|Tml^XBzNXytqK}$l~MZb zfa}q&b*xygsK>o_QML~Qe<5{B6Pm1_m_v*?aq*vUR7zr5)bYN#$FRe!cQ0MB_Fs>P zX&SXEFMJ2nXDmh}$y zoKVgQPBG)go=_nvKKcEzYWJB>FP_Nj-Wa8qe>eV3GAd!?rIHW9U+ z2MsD;^nB(d4q_d#kQ_~*4LUU2-`@1*x8nEk>sLB|He|^?NneHvZFC)sARA?xkB!3_ zKv!BAn}Pm04@0~jSAMdeD^S9=RWkH2y4v8c-?82xa^F- z*|39Pyv4iA;0lIZED8Ids^BiqO+}yV$le=lCdTcWmZ?16YYNnst8(aRV_5Iw%so-R zsFSyZGB2=YHqE6C$-7=I<0TJ(FGe5xjE9r*g&aeIEPl!pgj)!_#1h6F#dWR^88>uK ztae$Cw2tq0f0H=U-%VQLo?-$BzSp_key2BiN7JJ2jGmwgY-DV#5~>OgS8KHWvZ?!U z3{s`dKuS43&$_%RD9?(tJc6212WrcxGLYWQ7D&F--+H;&+~{b9E-07OmOYwkLPt3* zYjK{PH{lwK(Q~!NJ%=_H)n3Y>bzp47yA%YaxXf7#P&w3C4mDm>05mYKace)BD2B#C zhc7Ogjxk-pSVjg-2nU1VZHFV(KI?~jY<}C=-@(f^(eQg`3~qYT2^oHP)*rzoG~cz2vQDo8wSp; zzOz;Dmz@Uo8TZNU)%r5|DxQb(seq{<@Y@e+QCm%4Hh0xP+!DnQLrL9pFeA%h02h2> zuGp$uFfP!U4)Fc(X{6TM*2e|YB@(z%~7{TFsa^9v;Zbw`k?OQJMTi4{O*-7 z>@ggdHdI{ndE6_dLF(9nm;I!iNm%DfXoMQdv^-POkjJ$nWADqaBcl||a6V9;URq7v z6U}*XPSy)Iy?WjGQ)PT?_I8^JVI2G6j|zvMYy~+b;APA15vBAN;ehO;u}?5x+2yfW z0T`%Vl>y@fy|QUqEoOOZe+)ag5@0AHzjx797s!#c%5pVXFdb18X=R`0!`h5c0{13f zh#rm1?5~PckeWc0g^TS@vku#I%HB7)ZAB*vBfq`VPSyTY=rh*X*)fS1P6R}w0IN|- zMM@FUTbIhF?pQ(6TN?SyH#9=^b$!FfMZM2Ryq{E-xR~~F)(RYd9;@7aG|bQCKhS-| z(vM;)e&O9iPVI>Gn@UfAZpg-@0zj6$s`t}7^l4R1t9oa>ePY}le>{y?NRK_6a)-64 zHtNFaF=dOLU9R+D4_I4$gQ(}gF2|Pd40SH(k%EIQa!$&C%Udua%eQu+OY{3N0_*zs=c~e-i}hFM zgzh5HW$4u_L8&Ke2(t2fDuUl+yfrq*BBCm<0Qlihg*$FbAyjAG(i3)f7hmGe%Vc{3!f!Iyo2UjM7LEW2nku@C$=nF`2T-od;D-$raI+T`nf2Z84xLRmxjyu?8 zv!OIS>Fy{(fI4)zw02IyxQR`P5y;lMm?c=b8{l#2&l%@K4Qo4{#mF~%q4@NU(#vLd z5`E#=6+d6+vVzU?XAb}~92)63qjN_sX3h=0w`rLnqs18Y1~Yp$d#Qdr>0gO+d~W0m zDVfgW5P?;iEYZKNuuJPp@KQO*9N#~J23w(vf)KwAmNT%!$;}5$JjuUP-N_2PrQ~+J zuRA50mCN6z1;-<9WRVIQ_A+?+2@ZPhB{c8<>Y_S&EIjG{GyKu+qOdLB$czRwGmnU_ z(eJdCFKzG%x<8hfmXz2-rN|UFPq*d(-Pw< z1*50~Mh}_H&zrU~7K$254OA}Uc>RAuh!E-5sRfYN6_PB@0LE8o-ai+!C3^LEAd$ zCLpc0ird(7I-zU+`GgH9#RrtorN8>*iAhw;p~XStcJo|^9b1?r>b^i3yOtX=X;X!q*ljn zTx(VD!N$JbIxa>|$X>u3;)SK=)t|?Lqw}u5I0KS&#KL?FHyOSdd8ruzQ}QTQNG%yp z6s-hABwTZfUXkAj`_{~Q2;*FO5b$uQ>!mp;^@P?fAls4W?c(KSJ9Zxsv+O?UdC6&J zLds?@?r60W7df=-X=lOsDfFariW9})%c;@$-p{G<*&oruG)CZ=73ar zYJ@Rn?~XrjR7FuX&m%O=j#_CTh;-MGuyG#bDt^{sQ8GkA2vPiVz`|#ZsY#Y{i`j3>~AcMYG-u$uJ=s5dNcM4mAo4hda}M( z?qYgLpTAW)Yrw(z=`OiD^F+FUOI43y=og{b-R`+NYyL1y<^!>_JjIG zOBG;H96D1VkfpzVv(afX`^WceGFPUw--pjAc0v9w34M$Yw3_#;k03qsM3O^*yShtq zbI8rd>=<)lw$?3Lkv21Iww6S}GkI4^yL?`hV=LyWv{^snsSfkS`;lPR`7DFZ>`_7} z0f6KlW}pT|A(4xGytMpt?gt%UI_Q(gUzr~ga>e7AF3mf>X^Nn_N}iP>Dwnv)1#G_Q zm)(J>)&9;%q_P_+qE_v6?drVQkq3UWaen`0>*|yza1&BnRJ7QgR$JpdqTdyh?x#o` zx8O0`>^=!`?0MNUY2#+XX3+99eX!z%4s@&L+&lT*X^}0-IXF3#R|cU1eoL4R(SKM% z*-%cFhpuLwoqcX#R>=|A3Y%65sPS3%IBM|iUJ-oU@=1d1W*5DiU(4a|{K}}!ecEi$ zlx2lMt!XzqrFJVd^7|`P$emFYDr0*_`0oS;H6b}ubbt6%?dH?fj;`7af5Ki;ORTbj z#)H*%D1b0KC*)%BmiyLD?Sv<4p)aKKmIj&y`mV?mM&9tL*0Q#i*Ws5$(ORqhIOx#R zWH$$SZPmNYwC=vvv?yXNlDF+i(PAy&lgi0$CMDZ|!Z*H{K=6XbsQp^QXRJcHhjC6I z_WPLnu?#O+UOCT@{EQMo_!C9N`V#JuS>i#p8#jK~5Wxp6bTd-@K?xRBaT!o@S|{hE zw%!EQH2jJ8aq7*`_X{EG%KQG_b28^(<8EuiX*mn0&w^r=Ps`$tUGx1V%Ip~{q7Ck_ zmkkbgQv|A*&3&EH;z`p@^E7r`bKGynr#pJeM{!E#yMjvr@S-XW8=+!iP>5mwE`0rl z(92CDP@dF!E7*FXq@}1RZoQ-9VVV-dPBf=QMcAsOsL!5m^-KNT%ef5RK6>gw)T0yw zXQRF=B&7@6nC5MT`&DgA8T&f*c$6q{PNsDlm@sH`z}y=4y-aat|lFVy3u zie6_TYal^u4NB8?x8s$j+W6to{N4hGUB0Y}r&`5oFW_v# zb+8!T>WgXRDBFXM&Ql4)pTJLd^2=17AK3^6Q}lBYkxxMtd7kIU|8jkHk)HbT&ewR- zp!z1AG~>q?bSc!-i;;JxzlP;CXY@IO;l{iyzcMqIGhG{;974YYoCkTD zc*$MNv&#U6CSV^$N$C7S@Ekijnmk4R*07)=)p@_@@)uH>w4fsM)cpYVeGe4o@>;Kk zSMRfg$dttIjo`K&`$!i%RLj|j@_LBfP($tco_1_uD$-dRq0HQ1D3SMqAiR)wQ8)4} zIoB^Ds-~N}gtJGm=5O<0!OEki9s7q%HGALT(&w~13jqy3W@^&*oA8E%8|ry3IZ zhxrSg!mCK|as9c#-#5bp`{}p{1{Ap|{=OL=xI#e}w$l88@}E`-Ngm+`^U;`U#6PVP z5c%V~>RhW&n6EkuU}!mfjre_@q8XW5QHl86%xHztb)zKIB>mOPVl{@pbS*K9 z_D{J(NJKaCrwpOHElyar#4heAPyWy4aAzuds$>5_J{_~{`%CPDTcLi_;91RY!aDk) z?IBa>2K=Unf6#;^*E9;05C^(KPi}PG&0=+=x6J)0tWR8}f$?r~A zu+%??_ak1_rEZ6%u>U{Z=s(@illY?*_4|qhdNssPef}`6u*2)ve_bS@ e`~S2TaSBu=xzjKRAuj>`qo$($tm>&%#Qy+k``Qrz From 1d84d00bcf20c8f7cbf43c33a8ce293f8167a9be Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Mon, 25 May 2020 17:17:02 -0700 Subject: [PATCH 068/151] [BEAM-10077] using filename + hash instead of UUID for staging name --- .../core/construction/Environments.java | 23 ++++++++++------- .../beam/runners/dataflow/DataflowRunner.java | 12 +++------ .../runners/dataflow/util/PackageUtil.java | 25 ++++++------------- .../runners/dataflow/util/GCSUploadMain.java | 7 +++++- .../dataflow/util/PackageUtilTest.java | 24 ++++++++++-------- 5 files changed, 46 insertions(+), 45 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index c324b92e7f33..72ebd13b681c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.core.construction; +import com.fasterxml.jackson.core.Base64Variants; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; import java.io.FileOutputStream; @@ -215,25 +216,19 @@ public static List getArtifacts(List stagingFiles) Set deduplicatedStagingFiles = new LinkedHashSet<>(stagingFiles); for (String path : deduplicatedStagingFiles) { File file; - String stagedName; + String stagedName = null; if (path.contains("=")) { String[] components = path.split("=", 2); file = new File(components[1]); stagedName = components[0]; } else { file = new File(path); - stagedName = createStagingFileName(file); } // Spurious items get added to the classpath. Filter by just those that exist. if (file.exists()) { ArtifactInformation.Builder artifactBuilder = ArtifactInformation.newBuilder(); artifactBuilder.setTypeUrn(BeamUrns.getUrn(StandardArtifacts.Types.FILE)); artifactBuilder.setRoleUrn(BeamUrns.getUrn(StandardArtifacts.Roles.STAGING_TO)); - artifactBuilder.setRolePayload( - RunnerApi.ArtifactStagingToRolePayload.newBuilder() - .setStagedName(stagedName) - .build() - .toByteString()); HashCode hashCode; if (file.isDirectory()) { File zippedFile; @@ -264,6 +259,14 @@ public static List getArtifacts(List stagingFiles) .build() .toByteString()); } + if (stagedName == null) { + stagedName = createStagingFileName(file, hashCode); + } + artifactBuilder.setRolePayload( + RunnerApi.ArtifactStagingToRolePayload.newBuilder() + .setStagedName(stagedName) + .build() + .toByteString()); artifactsBuilder.add(artifactBuilder.build()); } } @@ -314,10 +317,12 @@ public static Set getJavaCapabilities() { return capabilities.build(); } - public static String createStagingFileName(File path) { + public static String createStagingFileName(File path, HashCode hash) { + String encodedHash = Base64Variants.MODIFIED_FOR_URL.encode(hash.asBytes()); + String fileName = Files.getNameWithoutExtension(path.getAbsolutePath()); String ext = path.isDirectory() ? "jar" : Files.getFileExtension(path.getAbsolutePath()); String suffix = Strings.isNullOrEmpty(ext) ? "" : "." + ext; - return UUID.randomUUID().toString() + suffix; + return String.format("%s-%s%s", fileName, encodedHash, suffix); } private static File zipDirectory(File directory) throws IOException { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 457634fe6d23..a6358c120435 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -55,7 +55,6 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; -import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -211,8 +210,8 @@ public class DataflowRunner extends PipelineRunner { @VisibleForTesting static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024; - @VisibleForTesting static final String PIPELINE_FILE_FORMAT = "pipeline-%s.pb"; - @VisibleForTesting static final String DATAFLOW_GRAPH_FILE_FORMAT = "dataflow_graph-%s.json"; + @VisibleForTesting static final String PIPELINE_FILE_NAME = "pipeline.pb"; + @VisibleForTesting static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.json"; private static final ObjectMapper MAPPER = new ObjectMapper(); @@ -893,10 +892,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { LOG.info("Staging pipeline description to {}", options.getStagingLocation()); byte[] serializedProtoPipeline = jobSpecification.getPipelineProto().toByteArray(); DataflowPackage stagedPipeline = - options - .getStager() - .stageToFile( - serializedProtoPipeline, String.format(PIPELINE_FILE_FORMAT, UUID.randomUUID())); + options.getStager().stageToFile(serializedProtoPipeline, PIPELINE_FILE_NAME); dataflowOptions.setPipelineUrl(stagedPipeline.getLocation()); if (!isNullOrEmpty(dataflowOptions.getDataflowWorkerJar())) { @@ -996,7 +992,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { .getStager() .stageToFile( DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8), - String.format(DATAFLOW_GRAPH_FILE_FORMAT, UUID.randomUUID())); + DATAFLOW_GRAPH_FILE_NAME); newJob.getSteps().clear(); newJob.setStepsLocation(stagedGraph.getLocation()); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index 0b5e47828dc7..72139931edfd 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -34,7 +34,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.UUID; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -52,7 +51,7 @@ import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.MoreFutures; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hasher; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.HashCode; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteSource; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Files; @@ -360,21 +359,15 @@ List stageClasspathElements( @AutoValue public abstract static class StagedFile { - public static PackageUtil.StagedFile of( - String source, String sha256, @Nullable String destination) { + public static PackageUtil.StagedFile of(String source, String sha256, String destination) { return new AutoValue_PackageUtil_StagedFile(source, sha256, destination); } - public static PackageUtil.StagedFile of(String source, String sha256) { - return new AutoValue_PackageUtil_StagedFile(source, sha256, null); - } - /** The file to stage. */ public abstract String getSource(); /** The SHA-256 hash of the source file. */ public abstract String getSha256(); /** Staged target for this file. */ - @Nullable public abstract String getDestination(); } @@ -405,25 +398,22 @@ public static PackageAttributes forFileToStage( } checkState(!file.isDirectory(), "Source file must not be a directory."); DataflowPackage destination = new DataflowPackage(); - String target = dest == null ? Environments.createStagingFileName(file) : dest; String resourcePath = FileSystems.matchNewResource(stagingPath, true) - .resolve(target, StandardResolveOptions.RESOLVE_FILE) + .resolve(dest, StandardResolveOptions.RESOLVE_FILE) .toString(); destination.setLocation(resourcePath); - destination.setName(target); + destination.setName(dest); return new AutoValue_PackageUtil_PackageAttributes( file, null, destination, file.length(), hash); } public static PackageAttributes forBytesToStage( byte[] bytes, String targetName, String stagingPath) { - - Hasher hasher = Hashing.sha256().newHasher(); - String hash = hasher.putBytes(bytes).hash().toString(); + HashCode hashCode = Hashing.sha256().newHasher().putBytes(bytes).hash(); long size = bytes.length; - String target = targetName == null ? UUID.randomUUID().toString() : targetName; + String target = Environments.createStagingFileName(new File(targetName), hashCode); String resourcePath = FileSystems.matchNewResource(stagingPath, true) @@ -433,7 +423,8 @@ public static PackageAttributes forBytesToStage( targetPackage.setName(target); targetPackage.setLocation(resourcePath); - return new AutoValue_PackageUtil_PackageAttributes(null, bytes, targetPackage, size, hash); + return new AutoValue_PackageUtil_PackageAttributes( + null, bytes, targetPackage, size, hashCode.toString()); } public PackageAttributes withPackageName(String overridePackageName) { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java index fb07652f38a1..468ec958334e 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/GCSUploadMain.java @@ -21,9 +21,11 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.stream.Collectors; +import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.HashCode; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Files; @@ -39,9 +41,12 @@ public static void main(String[] args) { .map( (String source) -> { try { + File file = new File(source); + HashCode hashCode = Files.asByteSource(file).hash(Hashing.sha256()); return PackageUtil.StagedFile.of( source, - Files.asByteSource(new File(source)).hash(Hashing.sha256()).toString()); + hashCode.toString(), + Environments.createStagingFileName(file, hashCode)); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java index 2847279babff..535fbce76355 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java @@ -66,6 +66,7 @@ import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.Environments; import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; import org.apache.beam.runners.dataflow.util.PackageUtil.StagedFile; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; @@ -85,6 +86,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.HashCode; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Files; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.LineReader; @@ -140,12 +142,12 @@ private File makeFileWithContents(String name, String contents) throws Exception private static PackageAttributes makePackageAttributes( File file, @Nullable String overridePackageName) throws IOException { - File sourceFile = file.isDirectory() ? zipDirectory(file) : file; + StagedFile stagedFile = makeStagedFile(file.getPath()); PackageAttributes attributes = PackageUtil.PackageAttributes.forFileToStage( - sourceFile.getPath(), - Files.asByteSource(sourceFile).hash(Hashing.sha256()).toString(), - null, + stagedFile.getSource(), + stagedFile.getSha256(), + stagedFile.getDestination(), STAGING_PATH); if (overridePackageName != null) { attributes = attributes.withPackageName(overridePackageName); @@ -160,15 +162,17 @@ private static StagedFile makeStagedFile(String source) throws IOException { private static StagedFile makeStagedFile(String source, String destName) throws IOException { File file = new File(source); File sourceFile; - String sha256; + HashCode hashCode; if (file.exists()) { sourceFile = file.isDirectory() ? zipDirectory(file) : file; - sha256 = Files.asByteSource(sourceFile).hash(Hashing.sha256()).toString(); + hashCode = Files.asByteSource(sourceFile).hash(Hashing.sha256()); } else { sourceFile = file; - sha256 = ""; + hashCode = Hashing.sha256().hashBytes(new byte[] {}); } - return StagedFile.of(sourceFile.getPath(), sha256, destName); + String destination = + destName == null ? Environments.createStagingFileName(file, hashCode) : destName; + return StagedFile.of(sourceFile.getPath(), hashCode.toString(), destination); } private static File zipDirectory(File directory) throws IOException { @@ -218,8 +222,8 @@ public void testPackageNamingWithFilesHavingSameContentsAndSameNames() throws Ex makeFileWithContents("folder2/folderA/sameName", "This is a test!"); DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDestination(); - assertNotEquals(target1.getName(), target2.getName()); - assertNotEquals(target1.getLocation(), target2.getLocation()); + assertEquals(target1.getName(), target2.getName()); + assertEquals(target1.getLocation(), target2.getLocation()); } @Test From 27f0dbae8080b93801ce56ef208470e41673062a Mon Sep 17 00:00:00 2001 From: rarokni Date: Tue, 26 May 2020 17:23:44 +0800 Subject: [PATCH 069/151] [BEAM-9770] BigQueryIO Patterns. Add Document update to snippit update. --- .../en/documentation/patterns/bigqueryio.md | 46 +++++++++++++++++++ .../en/documentation/patterns/overview.md | 3 ++ .../section-menu/en/documentation.html | 1 + 3 files changed, 50 insertions(+) create mode 100644 website/www/site/content/en/documentation/patterns/bigqueryio.md diff --git a/website/www/site/content/en/documentation/patterns/bigqueryio.md b/website/www/site/content/en/documentation/patterns/bigqueryio.md new file mode 100644 index 000000000000..49393b873b41 --- /dev/null +++ b/website/www/site/content/en/documentation/patterns/bigqueryio.md @@ -0,0 +1,46 @@ +--- +layout: section +title: "BigQuery patterns" +section_menu: section-menu/documentation.html +permalink: /documentation/patterns/bigqueryio/ +--- + + +# Google BigQuery patterns + +The samples on this page show you common patterns for use with BigQueryIO. + +{{< language-switcher java py >}} + +## BigQueryIO deadletter pattern +In production systems, it is useful to implement the deadletter pattern with BigQueryIO outputting any elements which had errors during processing by BigQueryIO into another PCollection for further processing. +The samples below print the errors, but in a production system they can be sent to a deadletter table for later correction. + +{{< paragraph class="language-java" >}} +When using `STREAMING_INSERTS` you can use the `WriteResult` object to access a `PCollection` with the `TableRows` that failed to be inserted into BigQuery. +If you also set the `withExtendedErrorInfo` property , you will be able to access a `PCollection` from the `WriteResult`. The `PCollection` will then include a reference to the table, the data row and the `InsertErrors`. Which errors are added to the deadletter queue is determined via the `InsertRetryPolicy`. +{{< /paragraph >}} + +{{< paragraph class="language-py" >}} +In the result tuple you can access `FailedRows` to access the failed inserts. +{{< /paragraph >}} + +{{< highlight java >}} +{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryIODeadLetter >}} +{{< /highlight >}} + +{{< highlight py >}} +{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" BigQueryIODeadLetter >}} +{{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/overview.md b/website/www/site/content/en/documentation/patterns/overview.md index a610c3c83243..561c8f00187c 100644 --- a/website/www/site/content/en/documentation/patterns/overview.md +++ b/website/www/site/content/en/documentation/patterns/overview.md @@ -35,6 +35,9 @@ Pipeline patterns demonstrate common Beam use cases. Pipeline patterns are based **Custom window patterns** - Patterns for windowing functions * [Using data to dynamically set session window gaps](/documentation/patterns/custom-windows/#using-data-to-dynamically-set-session-window-gaps) +**BigQuery patterns** - Patterns for BigQueryIO +* [Google BigQuery patterns](/documentation/patterns/bigqueryio/#google-bigquery-patterns) + ## Contributing a pattern To contribute a new pipeline pattern, create an issue with the [`pipeline-patterns` label](https://issues.apache.org/jira/browse/BEAM-7449?jql=labels%20%3D%20pipeline-patterns) and add details to the issue description. See [Get started contributing](/contribute/) for more information. diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index a44e0455a7d1..9baa95c26f6c 100644 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -276,6 +276,7 @@

  • Pipeline options
  • Custom I/O
  • Custom windows
  • +
  • BigQueryIO
  • From 79dcab9cab04b70d9e16f8c924fc52237f13622f Mon Sep 17 00:00:00 2001 From: Piotr Szuberski Date: Tue, 26 May 2020 13:26:53 +0200 Subject: [PATCH 070/151] [BEAM-10073] Add pubsub performance tests results to the grafana dashboards (#11809) --- ...ob_PerformanceTests_PubsubIO_Python.groovy | 5 +- .test-infra/metrics/beamgrafana-deploy.yaml | 2 +- .../Python_IO_IT_Tests_Dataflow.json | 256 +++++++++++++++++- 3 files changed, 255 insertions(+), 8 deletions(-) diff --git a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy index 5ca1458c4fcd..96361c007920 100644 --- a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy +++ b/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy @@ -39,7 +39,10 @@ def psio_test = [ temp_location : 'gs://temp-storage-for-perf-tests/loadtests', publish_to_big_query : true, metrics_dataset : 'beam_performance', - metrics_table : 'psio_io_2GB_msg_results', + metrics_table : 'psio_io_2GB_results', + influx_measurement : 'python_psio_2GB_results', + influx_db_name : InfluxDBCredentialsHelper.InfluxDBDatabaseName, + influx_hostname : InfluxDBCredentialsHelper.InfluxDBHostname, input_options : '\'{' + '"num_records": 2097152,' + '"key_size": 1,' + diff --git a/.test-infra/metrics/beamgrafana-deploy.yaml b/.test-infra/metrics/beamgrafana-deploy.yaml index 8a4ecb7a474a..7c5a83dd6784 100644 --- a/.test-infra/metrics/beamgrafana-deploy.yaml +++ b/.test-infra/metrics/beamgrafana-deploy.yaml @@ -37,7 +37,7 @@ spec: fsGroup: 1000 containers: - name: beamgrafana - image: gcr.io/apache-beam-testing/beamgrafana:beammetrics20200515 + image: gcr.io/apache-beam-testing/beamgrafana:beammetrics20200526 securityContext: runAsUser: 0 env: diff --git a/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json b/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json index 60d9a86dba56..d11465bc14fb 100644 --- a/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json +++ b/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json @@ -35,7 +35,7 @@ }, "hiddenSeries": false, "id": 2, - "interval": "", + "interval": "24h", "legend": { "avg": false, "current": false, @@ -75,7 +75,7 @@ "measurement": "python_bqio_read", "orderByTime": "ASC", "policy": "default", - "query": "SELECT value FROM \"python_bqio_read_10GB_results\" WHERE \"metric\" =~ /runtime/", + "query": "SELECT mean(\"value\") FROM \"python_bqio_read_10GB_results\" WHERE \"metric\" =~ /runtime/ AND $timeFilter GROUP BY time($__interval), \"metric\"", "rawQuery": true, "refId": "A", "resultFormat": "time_series", @@ -157,7 +157,7 @@ }, "hiddenSeries": false, "id": 3, - "interval": "", + "interval": "24h", "legend": { "avg": false, "current": false, @@ -197,7 +197,7 @@ "measurement": "python_bqio_read", "orderByTime": "ASC", "policy": "default", - "query": "SELECT value FROM \"python_bqio_write_10GB_results\" WHERE \"metric\" =~ /runtime/", + "query": "SELECT mean(\"value\") FROM \"python_bqio_write_10GB_results\" WHERE \"metric\" =~ /runtime/ AND $timeFilter GROUP BY time($__interval), \"metric\"", "rawQuery": true, "refId": "A", "resultFormat": "time_series", @@ -261,6 +261,250 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "BeamInfluxDB", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 12, + "x": 0, + "y": 9 + }, + "hiddenSeries": false, + "id": 4, + "interval": "24h", + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": false, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "connected", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.7.2", + "pointradius": 2, + "points": true, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "read_time", + "groupBy": [ + { + "params": [ + "$__interval" + ], + "type": "time" + } + ], + "measurement": "python_bqio_read", + "orderByTime": "ASC", + "policy": "default", + "query": "SELECT mean(\"value\") FROM \"python_psio_2GB_results\" WHERE \"metric\" = \"pubsub_io_perf_read_runtime\" AND $timeFilter GROUP BY time($__interval), \"metric\"", + "rawQuery": true, + "refId": "A", + "resultFormat": "time_series", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "field" + }, + { + "params": [], + "type": "mean" + } + ] + ], + "tags": [] + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Reading 2GB of data | Pubsub native Dataflow IO | streaming", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transparent": true, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:403", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:404", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "BeamInfluxDB", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 12, + "x": 12, + "y": 9 + }, + "hiddenSeries": false, + "id": 5, + "interval": "24h", + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": false, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "connected", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pluginVersion": "6.7.2", + "pointradius": 2, + "points": true, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "write_time", + "groupBy": [ + { + "params": [ + "$__interval" + ], + "type": "time" + } + ], + "measurement": "python_bqio_read", + "orderByTime": "ASC", + "policy": "default", + "query": "SELECT mean(\"value\") FROM \"python_psio_2GB_results\" WHERE \"metric\" = 'pubsub_io_perf_write_runtime' AND $timeFilter GROUP BY time($__interval), \"metric\"", + "rawQuery": true, + "refId": "A", + "resultFormat": "time_series", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "field" + }, + { + "params": [], + "type": "mean" + } + ] + ], + "tags": [] + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Writing 2GB of data | Pubsub native Dataflow IO | streaming", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transparent": true, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:403", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:404", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "schemaVersion": 22, @@ -293,5 +537,5 @@ "variables": { "list": [] }, - "version": 1 -} + "version": 2 +} \ No newline at end of file From fa028e68e47c2a47858783a5a5f7adc15569c654 Mon Sep 17 00:00:00 2001 From: darshan jani Date: Tue, 26 May 2020 22:36:35 +0800 Subject: [PATCH 071/151] [BEAM-10074] | implement hashing functions --- .../sql/impl/udf/BuiltinHashFunctions.java | 139 ++++++++++++++++++ .../sdk/extensions/sql/BeamSqlDslBase.java | 6 + .../BeamSalUhfSpecialTypeAndValueTest.java | 69 +++++++++ .../impl/udf/BeamSqlUdfExpressionTest.java | 41 ++++++ 4 files changed, 255 insertions(+) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java new file mode 100644 index 000000000000..c3fc82b1fdb5 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java @@ -0,0 +1,139 @@ +/* + * 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.beam.sdk.extensions.sql.impl.udf; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict; + +/** Hash Functions. */ +@AutoService(BeamBuiltinFunctionProvider.class) +public class BuiltinHashFunctions extends BeamBuiltinFunctionProvider { + + /** + * MD5(X) + * + *

    Calculates the MD5 digest and returns the value as a 16 element {@code byte[]}. + */ + @UDF( + funcName = "MD5", + parameterArray = {Schema.TypeName.STRING}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] md5String(String str) { + return org.apache.commons.codec.digest.DigestUtils.md5(str); + } + + /** + * MD5(X) + * + *

    Calculates the MD5 digest and returns the value as a 16 element {@code byte[]}. + */ + @UDF( + funcName = "MD5", + parameterArray = {Schema.TypeName.BYTES}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] md5Bytes(byte[] bytes) { + return org.apache.commons.codec.digest.DigestUtils.md5(bytes); + } + + /** + * SHA1(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA1", + parameterArray = {Schema.TypeName.STRING}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha1String(String str) { + return org.apache.commons.codec.digest.DigestUtils.sha1(str); + } + + /** + * SHA1(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA1", + parameterArray = {Schema.TypeName.BYTES}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha1Bytes(byte[] bytes) { + return org.apache.commons.codec.digest.DigestUtils.sha1(bytes); + } + + /** + * SHA256(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA256", + parameterArray = {Schema.TypeName.STRING}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha256String(String str) { + return org.apache.commons.codec.digest.DigestUtils.sha256(str); + } + + /** + * SHA256(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA256", + parameterArray = {Schema.TypeName.BYTES}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha256Bytes(byte[] bytes) { + return org.apache.commons.codec.digest.DigestUtils.sha256(bytes); + } + + /** + * SHA512(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA512", + parameterArray = {Schema.TypeName.STRING}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha512String(String str) { + return org.apache.commons.codec.digest.DigestUtils.sha512(str); + } + + /** + * SHA512(X) + * + *

    Calculates the SHA-1 digest and returns the value as a {@code byte[]}. + */ + @UDF( + funcName = "SHA512", + parameterArray = {Schema.TypeName.BYTES}, + returnType = Schema.TypeName.BYTES) + @Strict + public byte[] sha512Bytes(byte[] bytes) { + return org.apache.commons.codec.digest.DigestUtils.sha512(bytes); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java index ad26d4a27f5b..4298c0711cc4 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java @@ -213,6 +213,12 @@ public static void prepareClass() throws ParseException { "TO_HEX", "abcABC".getBytes(UTF_8), "TO_HEX", + "abcABCжщфЖЩФ".getBytes(UTF_8), + "HashingFn", + "foobar".getBytes(UTF_8), + "HashingFn", + " ".getBytes(UTF_8), + "HashingFn", "abcABCжщфЖЩФ".getBytes(UTF_8)) .getRows(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSalUhfSpecialTypeAndValueTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSalUhfSpecialTypeAndValueTest.java index 1370c62db0f9..db87eb77a203 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSalUhfSpecialTypeAndValueTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSalUhfSpecialTypeAndValueTest.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.commons.codec.digest.DigestUtils; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -167,4 +168,72 @@ public void testRightPad() throws Exception { resultRow8); pipeline.run().waitUntilFinish(); } + + @Test + public void testMd5() throws Exception { + Schema resultType = Schema.builder().addByteArrayField("field").build(); + Row resultRow1 = + Row.withSchema(resultType).addValues(DigestUtils.md5("foobar".getBytes(UTF_8))).build(); + Row resultRow2 = + Row.withSchema(resultType).addValues(DigestUtils.md5(" ".getBytes(UTF_8))).build(); + Row resultRow3 = + Row.withSchema(resultType) + .addValues(DigestUtils.md5("abcABCжщфЖЩФ".getBytes(UTF_8))) + .build(); + String sql = "SELECT MD5(f_bytes) FROM PCOLLECTION WHERE f_func = 'HashingFn'"; + PCollection result = boundedInputBytes.apply("testUdf", SqlTransform.query(sql)); + PAssert.that(result).containsInAnyOrder(resultRow1, resultRow2, resultRow3); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testSHA1() throws Exception { + Schema resultType = Schema.builder().addByteArrayField("field").build(); + Row resultRow1 = + Row.withSchema(resultType).addValues(DigestUtils.sha1("foobar".getBytes(UTF_8))).build(); + Row resultRow2 = + Row.withSchema(resultType).addValues(DigestUtils.sha1(" ".getBytes(UTF_8))).build(); + Row resultRow3 = + Row.withSchema(resultType) + .addValues(DigestUtils.sha1("abcABCжщфЖЩФ".getBytes(UTF_8))) + .build(); + String sql = "SELECT SHA1(f_bytes) FROM PCOLLECTION WHERE f_func = 'HashingFn'"; + PCollection result = boundedInputBytes.apply("testUdf", SqlTransform.query(sql)); + PAssert.that(result).containsInAnyOrder(resultRow1, resultRow2, resultRow3); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testSHA256() throws Exception { + Schema resultType = Schema.builder().addByteArrayField("field").build(); + Row resultRow1 = + Row.withSchema(resultType).addValues(DigestUtils.sha256("foobar".getBytes(UTF_8))).build(); + Row resultRow2 = + Row.withSchema(resultType).addValues(DigestUtils.sha256(" ".getBytes(UTF_8))).build(); + Row resultRow3 = + Row.withSchema(resultType) + .addValues(DigestUtils.sha256("abcABCжщфЖЩФ".getBytes(UTF_8))) + .build(); + String sql = "SELECT SHA256(f_bytes) FROM PCOLLECTION WHERE f_func = 'HashingFn'"; + PCollection result = boundedInputBytes.apply("testUdf", SqlTransform.query(sql)); + PAssert.that(result).containsInAnyOrder(resultRow1, resultRow2, resultRow3); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testSHA512() throws Exception { + Schema resultType = Schema.builder().addByteArrayField("field").build(); + Row resultRow1 = + Row.withSchema(resultType).addValues(DigestUtils.sha512("foobar".getBytes(UTF_8))).build(); + Row resultRow2 = + Row.withSchema(resultType).addValues(DigestUtils.sha512(" ".getBytes(UTF_8))).build(); + Row resultRow3 = + Row.withSchema(resultType) + .addValues(DigestUtils.sha512("abcABCжщфЖЩФ".getBytes(UTF_8))) + .build(); + String sql = "SELECT SHA512(f_bytes) FROM PCOLLECTION WHERE f_func = 'HashingFn'"; + PCollection result = boundedInputBytes.apply("testUdf", SqlTransform.query(sql)); + PAssert.that(result).containsInAnyOrder(resultRow1, resultRow2, resultRow3); + pipeline.run().waitUntilFinish(); + } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSqlUdfExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSqlUdfExpressionTest.java index 82ffccf9fb6f..495f32941f6f 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSqlUdfExpressionTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamSqlUdfExpressionTest.java @@ -21,6 +21,7 @@ import org.apache.beam.sdk.extensions.sql.integrationtest.BeamSqlBuiltinFunctionsIntegrationTestBase; import org.apache.beam.sdk.schemas.Schema.TypeName; +import org.apache.commons.codec.digest.DigestUtils; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -199,4 +200,44 @@ public void testRightPad() throws Exception { checker.buildRunAndCheck(); } + + @Test + public void testMd5() throws Exception { + ExpressionChecker checker = + new ExpressionChecker() + .addExpr("MD5('foobar')", DigestUtils.md5("foobar")) + .addExpr("MD5('中文')", DigestUtils.md5("中文")) + .addExprWithNullExpectedValue("MD5(CAST(NULL AS VARCHAR(0)))", TypeName.BYTES); + checker.buildRunAndCheck(); + } + + @Test + public void testSHA1() throws Exception { + ExpressionChecker checker = + new ExpressionChecker() + .addExpr("SHA1('foobar')", DigestUtils.sha1("foobar")) + .addExpr("SHA1('中文')", DigestUtils.sha1("中文")) + .addExprWithNullExpectedValue("SHA1(CAST(NULL AS VARCHAR(0)))", TypeName.BYTES); + checker.buildRunAndCheck(); + } + + @Test + public void testSHA256() throws Exception { + ExpressionChecker checker = + new ExpressionChecker() + .addExpr("SHA256('foobar')", DigestUtils.sha256("foobar")) + .addExpr("SHA256('中文')", DigestUtils.sha256("中文")) + .addExprWithNullExpectedValue("SHA256(CAST(NULL AS VARCHAR(0)))", TypeName.BYTES); + checker.buildRunAndCheck(); + } + + @Test + public void testSHA512() throws Exception { + ExpressionChecker checker = + new ExpressionChecker() + .addExpr("SHA512('foobar')", DigestUtils.sha512("foobar")) + .addExpr("SHA512('中文')", DigestUtils.sha512("中文")) + .addExprWithNullExpectedValue("SHA512(CAST(NULL AS VARCHAR(0)))", TypeName.BYTES); + checker.buildRunAndCheck(); + } } From f83bc7569864509a4ba2e7a5051fb0a4dd62281d Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Tue, 26 May 2020 12:42:13 -0400 Subject: [PATCH 072/151] [BEAM-10065] Fix beam release guide template (#11797) [BEAM-10065] Fix beam release guide template --- .../content/en/contribute/release-guide.md | 60 +++++++++---------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 81d39b4fa09b..4f949e2d4bd8 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -712,53 +712,53 @@ all major features and bug fixes, and all known issues. Template: ``` - We are happy to present the new {$RELEASE_VERSION} release of Beam. This release includes both improvements and new functionality. - See the [download page](/get-started/downloads/{$DOWNLOAD_ANCHOR}) for this release. - For more information on changes in {$RELEASE_VERSION}, check out the - [detailed release notes]({$JIRA_RELEASE_NOTES}). +We are happy to present the new {$RELEASE_VERSION} release of Beam. This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/{$DOWNLOAD_ANCHOR}) for this release. +For more information on changes in {$RELEASE_VERSION}, check out the +[detailed release notes]({$JIRA_RELEASE_NOTES}). - ## Highlights +## Highlights - * New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). - * New highly anticipated feature Y added to JavaSDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). + * New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). + * New highly anticipated feature Y added to JavaSDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). - {$TOPICS e.g.:} - ### I/Os - * Support for X source added (Java) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). - {$TOPICS} +{$TOPICS e.g.:} +### I/Os +* Support for X source added (Java) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +{$TOPICS} - ### New Features / Improvements +### New Features / Improvements - * X feature added (Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). - * Y feature added (Java) [BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y). +* X feature added (Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* Y feature added (Java) [BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y). - ### Breaking Changes +### Breaking Changes - * X behavior was changed ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). - * Y behavior was changed ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). +* X behavior was changed ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* Y behavior was changed ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). - ### Deprecations +### Deprecations - * X behavior is deprecated and will be removed in X versions ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* X behavior is deprecated and will be removed in X versions ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). - ### Bugfixes +### Bugfixes - * Fixed X (Python) ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-X)). - * Fixed Y (Java) ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). +* Fixed X (Python) ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-X)). +* Fixed Y (Java) ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). - ### Known Issues +### Known Issues - * {$KNOWN_ISSUE_1} - * {$KNOWN_ISSUE_2} - * See a full list of open [issues that affect](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20affectedVersion%20%3D%20{$RELEASE}%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC) this version. +* {$KNOWN_ISSUE_1} +* {$KNOWN_ISSUE_2} +* See a full list of open [issues that affect](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20affectedVersion%20%3D%20{$RELEASE}%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC) this version. - ## List of Contributors +## List of Contributors - According to git shortlog, the following people contributed to the 2.XX.0 release. Thank you to all contributors! +According to git shortlog, the following people contributed to the 2.XX.0 release. Thank you to all contributors! - ${CONTRIBUTORS} - ``` +${CONTRIBUTORS} +``` #### Checklist to proceed to the next step From f452e098242e7f4f48e5be7d34fe94039600f6c2 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Tue, 26 May 2020 14:04:42 -0400 Subject: [PATCH 073/151] Fix typo in doc "and and" -> "and" (#11810) --- sdks/python/apache_beam/io/textio.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 4099e87215e7..0660b8747f3a 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -438,7 +438,7 @@ def _create_text_source( class ReadAllFromText(PTransform): """A ``PTransform`` for reading a ``PCollection`` of text files. - Reads a ``PCollection`` of text files or file patterns and and produces a + Reads a ``PCollection`` of text files or file patterns and produces a ``PCollection`` of strings. Parses a text file as newline-delimited elements, by default assuming From d86219ffb478c7ecd130668858673cb5f6786c2e Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Tue, 26 May 2020 11:07:46 -0700 Subject: [PATCH 074/151] [BEAM-2939, BEAM-10057] Ensure that we can process an EmptyUnboundedSource and also prevent splitting on it. (#11781) --- .../src/main/java/org/apache/beam/sdk/io/Read.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index e02c938e68d6..19baa8d4cb0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -725,7 +725,7 @@ public void close() throws IOException {} @Override public Instant getWatermark() { - throw new UnsupportedOperationException("getWatermark is never meant to be invoked."); + return BoundedWindow.TIMESTAMP_MAX_VALUE; } @Override @@ -836,16 +836,20 @@ public UnboundedSourceRestriction currentRestriction() { @Override public SplitResult> trySplit( double fractionOfRemainder) { - // Don't split if we have claimed all since the SDF wrapper will be finishing soon. + // Don't split if we have the empty sources since the SDF wrapper will be finishing soon. + UnboundedSourceRestriction currentRestriction = currentRestriction(); + if (currentRestriction.getSource() instanceof EmptyUnboundedSource) { + return null; + } + // Our split result sets the primary to have no checkpoint mark associated // with it since when we resume we don't have any state but we specifically pass // the checkpoint mark to the current reader so that when we finish the current bundle // we may register for finalization. - UnboundedSourceRestriction currentRestriction = currentRestriction(); SplitResult> result = SplitResult.of( UnboundedSourceRestriction.create( - EmptyUnboundedSource.INSTANCE, null, currentRestriction.getWatermark()), + EmptyUnboundedSource.INSTANCE, null, BoundedWindow.TIMESTAMP_MAX_VALUE), currentRestriction); currentReader = EmptyUnboundedSource.INSTANCE.createReader(null, currentRestriction.getCheckpoint()); From 90a79d6c8e37a183678c9e3d0c046908334f2a48 Mon Sep 17 00:00:00 2001 From: Ashwin Ramaswami Date: Tue, 26 May 2020 18:41:55 -0400 Subject: [PATCH 075/151] [BEAM-7370] Upgrade sphinx to 3.0.3 (#11798) * Upgrade sphinx to 3.0.3 * update CHANGES.md --- CHANGES.md | 1 + sdks/python/tox.ini | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 59082149617a..7e57f487958b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,6 +65,7 @@ * `--direct_num_workers=0` is supported for FnApi runner. It will set the number of threads/subprocesses to number of cores of the machine executing the pipeline ([BEAM-9443](https://issues.apache.org/jira/browse/BEAM-9443)). * Python SDK now has experimental support for SqlTransform ([BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)). * Add OnWindowExpiration method to Stateful DoFn ([BEAM-1589](https://issues.apache.org/jira/browse/BEAM-1589)). +* Upgrade Sphinx to 3.0.3 for building PyDoc. ## Breaking Changes diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 3b66c28592a0..2726a74d39a1 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -200,7 +200,7 @@ commands = [testenv:py37-docs] extras = test,gcp,docs,interactive deps = - Sphinx==1.8.5 + Sphinx==3.0.3 sphinx_rtd_theme==0.4.3 commands = time {toxinidir}/scripts/generate_pydoc.sh From c77bee6b83f76775945e92bda9e9a8e295a2ce4a Mon Sep 17 00:00:00 2001 From: Yichi Zhang Date: Tue, 26 May 2020 15:43:22 -0700 Subject: [PATCH 076/151] Clear redundant debug statements in pr-11756 --- .../main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index ab07ca319ac0..0baf76b9dcdf 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -1049,9 +1049,6 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { Instant elementTimestampOrTimerFireTimestamp, PaneInfo paneInfo, TimeDomain timeDomain) { - if (timerId.equals("ts-event") && timeDomain.equals(TimeDomain.PROCESSING_TIME)) { - throw new IllegalStateException("illegal"); - } this.timerId = timerId; this.userKey = userKey; this.dynamicTimerTag = dynamicTimerTag; From 60220792034c5103760b5b6f605ab5795f61903b Mon Sep 17 00:00:00 2001 From: Steven Niemitz Date: Tue, 26 May 2020 23:04:29 -0400 Subject: [PATCH 077/151] [BEAM-10076] Fix dataflow worker status page rendering (#11812) --- .../beam/runners/dataflow/worker/StreamingDataflowWorker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 58aa30486d45..2f0871964685 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -2310,14 +2310,14 @@ public void printActiveWork(PrintWriter writer) { builder.append(""); builder.append(String.format("%016x", workItem.getShardingKey())); builder.append(""); - builder.append(workItem.getWorkToken()); + builder.append(String.format("%016x", workItem.getWorkToken())); builder.append(""); builder.append(queue.size() - 1); builder.append(""); builder.append(elapsedString(work.getStartTime(), now)); builder.append(""); builder.append(state); - builder.append("\n"); + builder.append(""); builder.append(elapsedString(work.getStateStartTime(), now)); builder.append("\n"); } From 630a4889bf9ebd5aaa9e21adfdcfabbe5f41beac Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Tue, 26 May 2020 21:49:50 -0700 Subject: [PATCH 078/151] [BEAM-10016, BEAM-10094] Disable single test to get portable validates runner signal back to green Follow-up on fixing the issue should happen separately. --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 +++++++ runners/flink/job-server/flink_job_server.gradle | 4 ++++ runners/spark/job-server/build.gradle | 4 ++++ 3 files changed, 15 insertions(+) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 534366ab40f2..8ba8a4d12bb4 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -257,6 +257,12 @@ class BeamModulePlugin implements Plugin { // includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner' // excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders' } + // Tests to include/exclude from running, by default all tests are included + Closure testFilter = { + // Use the following to include / exclude tests: + // includeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2' + // excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2' + } // Configuration for the classpath when running the test. Configuration testClasspathConfiguration // Additional system properties. @@ -1689,6 +1695,7 @@ class BeamModulePlugin implements Plugin { testClassesDirs = project.files(project.project(":sdks:java:core").sourceSets.test.output.classesDirs, project.project(":runners:core-java").sourceSets.test.output.classesDirs) maxParallelForks config.numParallelTests useJUnit(config.testCategories) + filter(config.testFilter) // increase maxHeapSize as this is directly correlated to direct memory, // see https://issues.apache.org/jira/browse/BEAM-6698 maxHeapSize = '4g' diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle index ed76e6b89365..0ce92a569dc0 100644 --- a/runners/flink/job-server/flink_job_server.gradle +++ b/runners/flink/job-server/flink_job_server.gradle @@ -163,6 +163,10 @@ def portableValidatesRunnerTask(String name, Boolean streaming) { excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs' excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo' }, + testFilter: { + // TODO(BEAM-10016) + excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2' + }, ) } diff --git a/runners/spark/job-server/build.gradle b/runners/spark/job-server/build.gradle index cc0628cbe595..26a141d9c0df 100644 --- a/runners/spark/job-server/build.gradle +++ b/runners/spark/job-server/build.gradle @@ -117,6 +117,10 @@ def portableValidatesRunnerTask(String name) { excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering' excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer' }, + testFilter: { + // TODO(BEAM-10094) + excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2' + }, ) } From e66734495586d860b163f6e1dad944f6a2f43e26 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 26 May 2020 22:49:51 -0700 Subject: [PATCH 079/151] Add a bare bones powered by page. --- .../site/content/en/community/powered-by.md | 25 +++++++++++++++++++ .../partials/section-menu/en/community.html | 1 + 2 files changed, 26 insertions(+) create mode 100644 website/www/site/content/en/community/powered-by.md diff --git a/website/www/site/content/en/community/powered-by.md b/website/www/site/content/en/community/powered-by.md new file mode 100644 index 000000000000..ad22039388b9 --- /dev/null +++ b/website/www/site/content/en/community/powered-by.md @@ -0,0 +1,25 @@ +--- +title: 'Powered by Apache Beam' +--- + +# Projects and Products Powered by Apache Beam + +To add yourself to the list, please open a [pull request](https://github.com/apache/beam/edit/master/website/www/site/content/en/community/powered_by.md) adding your organization name, URL, a list of which Beam components you are using, and a short description of your use case. + +* **[Cloud Dataflow](https://cloud.google.com/dataflow):** Google Cloud Dataflow is a fully managed service for executing + Apache Beam pipelines within the Google Cloud Platform ecosystem. +* **[TensorFlow Extended (TFX)](https://www.tensorflow.org/tfx):** TensorFlow Extended (TFX) is an end-to-end platform + for deploying production ML pipelines. + diff --git a/website/www/site/layouts/partials/section-menu/en/community.html b/website/www/site/layouts/partials/section-menu/en/community.html index b2cafee008f7..d90582ffe861 100644 --- a/website/www/site/layouts/partials/section-menu/en/community.html +++ b/website/www/site/layouts/partials/section-menu/en/community.html @@ -14,6 +14,7 @@

  • Integrations
  • Contact Us
  • Policies
  • +
  • Powered by Apache Beam
  • YouTube channel
  • Twitter Handle
  • In Person
  • From 292dea3670853c34aca2ef80089e4bd31a0134eb Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Thu, 14 May 2020 17:31:38 +0200 Subject: [PATCH 080/151] [BEAM-9936] Reduce the code duplication across python/container/py* --- sdks/python/container/common.gradle | 71 +++++++++++++++++++++++++ sdks/python/container/py2/build.gradle | 53 +----------------- sdks/python/container/py35/build.gradle | 53 ++---------------- sdks/python/container/py36/build.gradle | 53 ++---------------- sdks/python/container/py37/build.gradle | 53 ++---------------- 5 files changed, 82 insertions(+), 201 deletions(-) create mode 100644 sdks/python/container/common.gradle diff --git a/sdks/python/container/common.gradle b/sdks/python/container/common.gradle new file mode 100644 index 000000000000..2a6ec55882d2 --- /dev/null +++ b/sdks/python/container/common.gradle @@ -0,0 +1,71 @@ +/* + * 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. + */ + +def pythonVersionSuffix = project.ext.pythonVersion.replace('.', '') + +description = "Apache Beam :: SDKs :: Python :: Container :: Python ${pythonVersionSuffix} Container" + +configurations { + sdkSourceTarball + sdkHarnessLauncher +} + +dependencies { + sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") + sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") +} + +task copyDockerfileDependencies(type: Copy) { + from configurations.sdkSourceTarball + from file("../base_image_requirements.txt") + into "build/target" + if(configurations.sdkSourceTarball.isEmpty()) { + throw new StopExecutionException(); + } +} + +task copyLicenseScripts(type: Copy){ + from ("../license_scripts") + into "build/target/license_scripts" +} + +task copyLauncherDependencies(type: Copy) { + from configurations.sdkHarnessLauncher + into "build/target/launcher" + if(configurations.sdkHarnessLauncher.isEmpty()) { + throw new StopExecutionException(); + } +} + +docker { + name containerImageName( + name: project.docker_image_default_repo_prefix + "python${project.ext.pythonVersion}_sdk", + root: project.rootProject.hasProperty(["docker-repository-root"]) ? + project.rootProject["docker-repository-root"] : + project.docker_image_default_repo_root, + tag: project.rootProject.hasProperty(["docker-tag"]) ? + project.rootProject["docker-tag"] : project.sdk_version) + files "../Dockerfile", "./build" + buildArgs(['py_version': "${project.ext.pythonVersion}", + 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || + project.rootProject.hasProperty(["isRelease"])]) +} + +dockerPrepare.dependsOn copyLauncherDependencies +dockerPrepare.dependsOn copyDockerfileDependencies +dockerPrepare.dependsOn copyLicenseScripts diff --git a/sdks/python/container/py2/build.gradle b/sdks/python/container/py2/build.gradle index 2821600419d7..6d3c94af3cea 100644 --- a/sdks/python/container/py2/build.gradle +++ b/sdks/python/container/py2/build.gradle @@ -21,55 +21,6 @@ plugins { id 'org.apache.beam.module' } applyDockerNature() +applyPythonNature() -description = "Apache Beam :: SDKs :: Python :: Container :: Python 2 Container" - -configurations { - sdkSourceTarball - sdkHarnessLauncher -} - -dependencies { - sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") - sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") -} - -task copyDockerfileDependencies(type: Copy) { - from configurations.sdkSourceTarball - from file("../base_image_requirements.txt") - into "build/target" - if(configurations.sdkSourceTarball.isEmpty()) { - throw new StopExecutionException(); - } -} - -task copyLicenseScripts(type: Copy){ - from ("../license_scripts") - into "build/target/license_scripts" -} - -task copyLauncherDependencies(type: Copy) { - from configurations.sdkHarnessLauncher - into "build/target/launcher" - if(configurations.sdkHarnessLauncher.isEmpty()) { - throw new StopExecutionException(); - } -} - -docker { - name containerImageName( - name: project.docker_image_default_repo_prefix + "python2.7_sdk", - root: project.rootProject.hasProperty(["docker-repository-root"]) ? - project.rootProject["docker-repository-root"] : - project.docker_image_default_repo_root, - tag: project.rootProject.hasProperty(["docker-tag"]) ? - project.rootProject["docker-tag"] : project.sdk_version) - files "../Dockerfile", "./build" - buildArgs(['py_version': "2.7", - 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"])]) -} - -dockerPrepare.dependsOn copyLauncherDependencies -dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScripts +apply from: "../common.gradle" diff --git a/sdks/python/container/py35/build.gradle b/sdks/python/container/py35/build.gradle index 6a0133dd9916..25aa4f92e14c 100644 --- a/sdks/python/container/py35/build.gradle +++ b/sdks/python/container/py35/build.gradle @@ -21,55 +21,8 @@ plugins { id 'org.apache.beam.module' } applyDockerNature() +applyPythonNature() -description = "Apache Beam :: SDKs :: Python :: Container :: Python 35 Container" +pythonVersion = '3.5' -configurations { - sdkSourceTarball - sdkHarnessLauncher -} - -dependencies { - sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") - sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") -} - -task copyDockerfileDependencies(type: Copy) { - from configurations.sdkSourceTarball - from file("../base_image_requirements.txt") - into "build/target" - if(configurations.sdkSourceTarball.isEmpty()) { - throw new StopExecutionException(); - } -} - -task copyLicenseScripts(type: Copy){ - from ("../license_scripts") - into "build/target/license_scripts" -} - -task copyLauncherDependencies(type: Copy) { - from configurations.sdkHarnessLauncher - into "build/target/launcher" - if(configurations.sdkHarnessLauncher.isEmpty()) { - throw new StopExecutionException(); - } -} - -docker { - name containerImageName( - name: project.docker_image_default_repo_prefix + "python3.5_sdk", - root: project.rootProject.hasProperty(["docker-repository-root"]) ? - project.rootProject["docker-repository-root"] : - project.docker_image_default_repo_root, - tag: project.rootProject.hasProperty(["docker-tag"]) ? - project.rootProject["docker-tag"] : project.sdk_version) - files "../Dockerfile", "./build" - buildArgs(['py_version': "3.5", - 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"])]) -} - -dockerPrepare.dependsOn copyLauncherDependencies -dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScripts +apply from: "../common.gradle" diff --git a/sdks/python/container/py36/build.gradle b/sdks/python/container/py36/build.gradle index b50267c29201..20fe8755bf50 100644 --- a/sdks/python/container/py36/build.gradle +++ b/sdks/python/container/py36/build.gradle @@ -21,55 +21,8 @@ plugins { id 'org.apache.beam.module' } applyDockerNature() +applyPythonNature() -description = "Apache Beam :: SDKs :: Python :: Container :: Python 36 Container" +pythonVersion = '3.6' -configurations { - sdkSourceTarball - sdkHarnessLauncher -} - -dependencies { - sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") - sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") -} - -task copyDockerfileDependencies(type: Copy) { - from configurations.sdkSourceTarball - from file("../base_image_requirements.txt") - into "build/target" - if(configurations.sdkSourceTarball.isEmpty()) { - throw new StopExecutionException(); - } -} - -task copyLicenseScripts(type: Copy){ - from ("../license_scripts") - into "build/target/license_scripts" -} - -task copyLauncherDependencies(type: Copy) { - from configurations.sdkHarnessLauncher - into "build/target/launcher" - if(configurations.sdkHarnessLauncher.isEmpty()) { - throw new StopExecutionException(); - } -} - -docker { - name containerImageName( - name: project.docker_image_default_repo_prefix + "python3.6_sdk", - root: project.rootProject.hasProperty(["docker-repository-root"]) ? - project.rootProject["docker-repository-root"] : - project.docker_image_default_repo_root, - tag: project.rootProject.hasProperty(["docker-tag"]) ? - project.rootProject["docker-tag"] : project.sdk_version) - files "../Dockerfile", "./build" - buildArgs(['py_version': "3.6", - 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"])]) -} - -dockerPrepare.dependsOn copyLauncherDependencies -dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScripts +apply from: "../common.gradle" diff --git a/sdks/python/container/py37/build.gradle b/sdks/python/container/py37/build.gradle index 969a58db7945..547163a3514e 100644 --- a/sdks/python/container/py37/build.gradle +++ b/sdks/python/container/py37/build.gradle @@ -21,55 +21,8 @@ plugins { id 'org.apache.beam.module' } applyDockerNature() +applyPythonNature() -description = "Apache Beam :: SDKs :: Python :: Container :: Python 37 Container" +pythonVersion = '3.7' -configurations { - sdkSourceTarball - sdkHarnessLauncher -} - -dependencies { - sdkSourceTarball project(path: ":sdks:python", configuration: "distTarBall") - sdkHarnessLauncher project(path: ":sdks:python:container", configuration: "sdkHarnessLauncher") -} - -task copyDockerfileDependencies(type: Copy) { - from configurations.sdkSourceTarball - from file("../base_image_requirements.txt") - into "build/target" - if(configurations.sdkSourceTarball.isEmpty()) { - throw new StopExecutionException(); - } -} - -task copyLicenseScripts(type: Copy){ - from ("../license_scripts") - into "build/target/license_scripts" -} - -task copyLauncherDependencies(type: Copy) { - from configurations.sdkHarnessLauncher - into "build/target/launcher" - if(configurations.sdkHarnessLauncher.isEmpty()) { - throw new StopExecutionException(); - } -} - -docker { - name containerImageName( - name: project.docker_image_default_repo_prefix + "python3.7_sdk", - root: project.rootProject.hasProperty(["docker-repository-root"]) ? - project.rootProject["docker-repository-root"] : - project.docker_image_default_repo_root, - tag: project.rootProject.hasProperty(["docker-tag"]) ? - project.rootProject["docker-tag"] : project.sdk_version) - files "../Dockerfile", "./build" - buildArgs(['py_version': "3.7", - 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || - project.rootProject.hasProperty(["isRelease"])]) -} - -dockerPrepare.dependsOn copyLauncherDependencies -dockerPrepare.dependsOn copyDockerfileDependencies -dockerPrepare.dependsOn copyLicenseScripts +apply from: "../common.gradle" From 09a1677906cae3a56a76c67e3e622565a1db408a Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Tue, 19 May 2020 12:22:24 +0200 Subject: [PATCH 081/151] [BEAM-9936] Create SDK harness containers with Python3.8 --- ..._Python_ValidatesContainer_Dataflow.groovy | 2 ++ build.gradle | 1 + sdks/python/container/Dockerfile | 2 +- .../container/base_image_requirements.txt | 12 ++++---- sdks/python/container/build.gradle | 1 + .../license_scripts/dep_urls_py.yaml | 4 ++- sdks/python/container/py38/build.gradle | 28 +++++++++++++++++++ .../container/run_validatescontainer.sh | 20 +++++++------ settings.gradle | 1 + 9 files changed, 54 insertions(+), 17 deletions(-) create mode 100644 sdks/python/container/py38/build.gradle diff --git a/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy index f25f133c282e..bd5888f42df3 100644 --- a/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy +++ b/.test-infra/jenkins/job_PostCommit_Python_ValidatesContainer_Dataflow.groovy @@ -39,5 +39,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Py_ValCont', shell('cd ' + commonJobProperties.checkoutDir + ' && bash sdks/python/container/run_validatescontainer.sh python35') shell('cd ' + commonJobProperties.checkoutDir + ' && bash sdks/python/container/run_validatescontainer.sh python36') shell('cd ' + commonJobProperties.checkoutDir + ' && bash sdks/python/container/run_validatescontainer.sh python37') + // TODO(BEAM-9754): Turn on ValidatesContainer tests on Python 3.8 once BEAM-9754 is resolved. + // shell('cd ' + commonJobProperties.checkoutDir + ' && bash sdks/python/container/run_validatescontainer.sh python38') } } diff --git a/build.gradle b/build.gradle index 65363b7569dc..8de743ab983c 100644 --- a/build.gradle +++ b/build.gradle @@ -229,6 +229,7 @@ task pythonDockerBuildPreCommit() { dependsOn ":sdks:python:container:py35:docker" dependsOn ":sdks:python:container:py36:docker" dependsOn ":sdks:python:container:py37:docker" + dependsOn ":sdks:python:container:py38:docker" } task pythonLintPreCommit() { diff --git a/sdks/python/container/Dockerfile b/sdks/python/container/Dockerfile index 7af872384c18..f28d6e3b34a0 100644 --- a/sdks/python/container/Dockerfile +++ b/sdks/python/container/Dockerfile @@ -17,7 +17,7 @@ ############################################################################### ARG py_version -FROM python:"${py_version}"-stretch +FROM python:"${py_version}"-buster MAINTAINER "Apache Beam " ARG pull_licenses diff --git a/sdks/python/container/base_image_requirements.txt b/sdks/python/container/base_image_requirements.txt index f3d9b8622118..4c3fd767cb06 100644 --- a/sdks/python/container/base_image_requirements.txt +++ b/sdks/python/container/base_image_requirements.txt @@ -25,18 +25,18 @@ avro==1.8.2;python_version<="2.7" avro-python3==1.8.2;python_version>="3.4" -fastavro==0.21.24 +fastavro==0.23.3 crcmod==1.7 dill==0.3.1.1 -future==0.17.1 +future==0.18.2 futures==3.2.0;python_version<"3.0" grpcio==1.24.3 hdfs==2.5.8 httplib2==0.12.0 mock==2.0.0 oauth2client==3.0.0 -protobuf==3.11.1 -pyarrow==0.15.1 +protobuf==3.12.0 +pyarrow==0.16.0 pydot==1.4.1 pytz==2019.3 pyvcf==0.6.8;python_version<"3.0" @@ -66,8 +66,8 @@ pandas==0.24.2;python_version<="2.7" pandas==0.25.2;python_version>="3.4" protorpc==0.12.0 python-gflags==3.0.6 - -tensorflow==2.1.0 +tensorflow==2.1.0;python_version<="2.7" +tensorflow==2.2.0;python_version>="3.5" pymongo==3.9.0 # Packages needed for testing. diff --git a/sdks/python/container/build.gradle b/sdks/python/container/build.gradle index 2f7662caf3a7..f759088fbbbf 100644 --- a/sdks/python/container/build.gradle +++ b/sdks/python/container/build.gradle @@ -55,6 +55,7 @@ task buildAll { dependsOn ':sdks:python:container:py35:docker' dependsOn ':sdks:python:container:py36:docker' dependsOn ':sdks:python:container:py37:docker' + dependsOn ':sdks:python:container:py38:docker' } artifacts { diff --git a/sdks/python/container/license_scripts/dep_urls_py.yaml b/sdks/python/container/license_scripts/dep_urls_py.yaml index 4a2b6f355736..d1e194f0add1 100644 --- a/sdks/python/container/license_scripts/dep_urls_py.yaml +++ b/sdks/python/container/license_scripts/dep_urls_py.yaml @@ -114,7 +114,9 @@ pip_dependencies: license: "https://raw.githubusercontent.com/tensorflow/tensorflow/master/LICENSE" tensorflow-estimator: license: "https://raw.githubusercontent.com/tensorflow/estimator/master/LICENSE" + tensorboard-plugin-wit: + license: "https://raw.githubusercontent.com/PAIR-code/what-if-tool/master/LICENSE" timeloop: license: "https://raw.githubusercontent.com/sankalpjonn/timeloop/master/LICENSE" wget: - license: "https://raw.githubusercontent.com/mirror/wget/master/COPYING" \ No newline at end of file + license: "https://raw.githubusercontent.com/mirror/wget/master/COPYING" diff --git a/sdks/python/container/py38/build.gradle b/sdks/python/container/py38/build.gradle new file mode 100644 index 000000000000..304895a83718 --- /dev/null +++ b/sdks/python/container/py38/build.gradle @@ -0,0 +1,28 @@ +/* + * 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. + */ + +plugins { + id 'base' + id 'org.apache.beam.module' +} +applyDockerNature() +applyPythonNature() + +pythonVersion = '3.8' + +apply from: "../common.gradle" diff --git a/sdks/python/container/run_validatescontainer.sh b/sdks/python/container/run_validatescontainer.sh index 2b48344ff149..330759d71d60 100755 --- a/sdks/python/container/run_validatescontainer.sh +++ b/sdks/python/container/run_validatescontainer.sh @@ -19,22 +19,23 @@ # This script will be run by Jenkins as a post commit test. In order to run # locally make the following changes: # -# LOCAL_PATH -> Path of tox and virtualenv if you have them already installed. # GCS_LOCATION -> Temporary location to use for service tests. # PROJECT -> Project name to use for dataflow and docker images. +# REGION -> Region name to use for Dataflow # # Execute from the root of the repository: # test Python2 container: ./sdks/python/container/run_validatescontainer.sh python2 # test Python3 container: ./sdks/python/container/run_validatescontainer.sh python35 # test Python3 container: ./sdks/python/container/run_validatescontainer.sh python36 # test Python3 container: ./sdks/python/container/run_validatescontainer.sh python37 +# test Python3 container: ./sdks/python/container/run_validatescontainer.sh python38 -echo "This script must be executed in the root of beam project. Please set LOCAL_PATH, GCS_LOCATION, and PROJECT as desired." +echo "This script must be executed in the root of beam project. Please set GCS_LOCATION, PROJECT and REGION as desired." if [[ $# != 1 ]]; then printf "Usage: \n$> ./sdks/python/container/run_validatescontainer.sh " printf "\n\tpython_version: [required] Python version used for container build and run tests." - printf " Use 'python2' for Python2, 'python35' for Python3.5, python36 for Python3.6, python37 for Python3.7." + printf " Use 'python2' for Python2, 'python35' for Python3.5, python36 for Python3.6, python37 for Python3.7, python38 for Python3.8." exit 1 fi @@ -46,31 +47,32 @@ GCS_LOCATION=${GCS_LOCATION:-gs://temp-storage-for-end-to-end-tests} # Project for the container and integration test PROJECT=${PROJECT:-apache-beam-testing} +REGION=${REGION:-us-central1} IMAGE_PREFIX="$(grep 'docker_image_default_repo_prefix' gradle.properties | cut -d'=' -f2)" # Other variables branched by Python version. if [[ $1 == "python2" ]]; then IMAGE_NAME="${IMAGE_PREFIX}python2.7_sdk" # Use this to create CONTAINER_IMAGE variable. CONTAINER_PROJECT="sdks:python:container:py2" # Use this to build container by Gradle. - GRADLE_PY3_FLAG="" # Use this in Gradle command. PY_INTERPRETER="python" # Use this in virtualenv command. elif [[ $1 == "python35" ]]; then IMAGE_NAME="${IMAGE_PREFIX}python3.5_sdk" # Use this to create CONTAINER_IMAGE variable. CONTAINER_PROJECT="sdks:python:container:py35" # Use this to build container by Gradle. - GRADLE_PY3_FLAG="-Ppython3" # Use this in Gradle command. PY_INTERPRETER="python3.5" # Use this in virtualenv command. elif [[ $1 == "python36" ]]; then IMAGE_NAME="${IMAGE_PREFIX}python3.6_sdk" # Use this to create CONTAINER_IMAGE variable. CONTAINER_PROJECT="sdks:python:container:py36" # Use this to build container by Gradle. - GRADLE_PY3_FLAG="-Ppython3" # Use this in Gradle command. PY_INTERPRETER="python3.6" # Use this in virtualenv command. elif [[ $1 == "python37" ]]; then IMAGE_NAME="${IMAGE_PREFIX}python3.7_sdk" # Use this to create CONTAINER_IMAGE variable. CONTAINER_PROJECT="sdks:python:container:py37" # Use this to build container by Gradle. - GRADLE_PY3_FLAG="-Ppython3" # Use this in Gradle command. PY_INTERPRETER="python3.7" # Use this in virtualenv command. +elif [[ $1 == "python38" ]]; then + IMAGE_NAME="${IMAGE_PREFIX}python3.8_sdk" # Use this to create CONTAINER_IMAGE variable. + CONTAINER_PROJECT="sdks:python:container:py38" # Use this to build container by Gradle. + PY_INTERPRETER="python3.8" # Use this in virtualenv command. else - echo "Must set Python version with one of 'python2', 'python35', 'python36' and 'python37' from commandline." + echo "Must set Python version with one of 'python2', 'python35', 'python36', 'python37' and 'python38' from commandline." exit 1 fi XUNIT_FILE="nosetests-$IMAGE_NAME.xml" @@ -88,7 +90,7 @@ gcloud -v TAG=$(date +%Y%m%d-%H%M%S) CONTAINER=us.gcr.io/$PROJECT/$USER/$IMAGE_NAME echo "Using container $CONTAINER" -./gradlew :$CONTAINER_PROJECT:docker -Pdocker-repository-root=us.gcr.io/$PROJECT/$USER -Pdocker-tag=$TAG $GRADLE_PY3_FLAG --info +./gradlew :$CONTAINER_PROJECT:docker -Pdocker-repository-root=us.gcr.io/$PROJECT/$USER -Pdocker-tag=$TAG --info # Verify it exists docker images | grep $TAG diff --git a/settings.gradle b/settings.gradle index 8a80ffd06298..36493decd0e3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -138,6 +138,7 @@ include ":sdks:python:container:py2" include ":sdks:python:container:py35" include ":sdks:python:container:py36" include ":sdks:python:container:py37" +include ":sdks:python:container:py38" include ":sdks:python:test-suites:dataflow:py2" include ":sdks:python:test-suites:dataflow:py35" include ":sdks:python:test-suites:dataflow:py36" From 6654a3daab11bb9428482df9c68cfcee03deadd8 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Wed, 13 May 2020 16:33:43 +0200 Subject: [PATCH 082/151] [BEAM-9810] Tox suites for Python 3.8 --- build.gradle | 3 +- sdks/python/test-suites/tox/py37/build.gradle | 3 -- sdks/python/test-suites/tox/py38/build.gradle | 35 +++++++++++++++++++ .../test-suites/tox/pycommon/build.gradle | 2 +- sdks/python/tox.ini | 27 ++++++++++++-- settings.gradle | 1 + 6 files changed, 64 insertions(+), 7 deletions(-) create mode 100644 sdks/python/test-suites/tox/py38/build.gradle diff --git a/build.gradle b/build.gradle index 65363b7569dc..f8fd0a1a0072 100644 --- a/build.gradle +++ b/build.gradle @@ -215,6 +215,7 @@ task pythonPreCommit() { dependsOn ":sdks:python:test-suites:tox:py35:preCommitPy35" dependsOn ":sdks:python:test-suites:tox:py36:preCommitPy36" dependsOn ":sdks:python:test-suites:tox:py37:preCommitPy37" + dependsOn ":sdks:python:test-suites:tox:py38:preCommitPy38" dependsOn ":sdks:python:test-suites:dataflow:py2:preCommitIT" dependsOn ":sdks:python:test-suites:dataflow:py2:preCommitIT_V2" dependsOn ":sdks:python:test-suites:dataflow:py37:preCommitIT" @@ -237,7 +238,7 @@ task pythonLintPreCommit() { } task pythonFormatterPreCommit() { - dependsOn 'sdks:python:test-suites:tox:py37:formatter' + dependsOn 'sdks:python:test-suites:tox:py38:formatter' } task python2PostCommit() { diff --git a/sdks/python/test-suites/tox/py37/build.gradle b/sdks/python/test-suites/tox/py37/build.gradle index f9126da48903..2583d385bc50 100644 --- a/sdks/python/test-suites/tox/py37/build.gradle +++ b/sdks/python/test-suites/tox/py37/build.gradle @@ -35,9 +35,6 @@ lint.dependsOn lintPy37 toxTask "mypyPy37", "py37-mypy" lint.dependsOn mypyPy37 -toxTask "formatter", "py3-yapf-check" -check.dependsOn formatter - apply from: "../common.gradle" // TODO(BEAM-8954): Remove this once tox uses isolated builds. diff --git a/sdks/python/test-suites/tox/py38/build.gradle b/sdks/python/test-suites/tox/py38/build.gradle new file mode 100644 index 000000000000..96b02f805c25 --- /dev/null +++ b/sdks/python/test-suites/tox/py38/build.gradle @@ -0,0 +1,35 @@ +/* + * 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. + */ + +/** + * Unit tests for Python 3.8 + */ + +plugins { id 'org.apache.beam.module' } +applyPythonNature() + +// Required to setup a Python 3 virtualenv and task names. +pythonVersion = '3.8' + +toxTask "formatter", "py3-yapf-check" +check.dependsOn formatter + +apply from: "../common.gradle" + +// TODO(BEAM-8954): Remove this once tox uses isolated builds. +testPy38Cython.mustRunAfter testPython38, testPy38Cloud diff --git a/sdks/python/test-suites/tox/pycommon/build.gradle b/sdks/python/test-suites/tox/pycommon/build.gradle index 80ad2d4175f0..80342ce990d5 100644 --- a/sdks/python/test-suites/tox/pycommon/build.gradle +++ b/sdks/python/test-suites/tox/pycommon/build.gradle @@ -23,7 +23,7 @@ plugins { id 'org.apache.beam.module' } applyPythonNature() -toxTask "docs", "py37-docs" +toxTask "docs", "py38-docs" assemble.dependsOn docs task preCommitPyCommon() { diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 2726a74d39a1..745ed926907a 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -17,7 +17,7 @@ [tox] # new environments will be excluded by default unless explicitly added to envlist. -envlist = py27,py35,py36,py37,py27-{cloud,cython,lint,lint3},py35-{cloud,cython},py36-{cloud,cython},py37-{cloud,cython,lint,mypy,docs} +envlist = py27,py35,py36,py37,py27-{cloud,cython,lint,lint3},py35-{cloud,cython},py36-{cloud,cython},py37-{cloud,cython,lint,mypy},py38-{cloud,cython,docs} toxworkdir = {toxinidir}/target/{env:ENV_NAME:.tox} [pycodestyle] @@ -76,6 +76,11 @@ commands = python apache_beam/examples/complete/autocomplete_test.py {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" +[testenv:py38] +commands = + python apache_beam/examples/complete/autocomplete_test.py + {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + [testenv:py27-cython] # cython tests are only expected to work in linux (2.x and 3.x) # If we want to add other platforms in the future, it should be: @@ -128,6 +133,19 @@ commands = python apache_beam/examples/complete/autocomplete_test.py {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" +[testenv:py38-cython] +# cython tests are only expected to work in linux (2.x and 3.x) +# If we want to add other platforms in the future, it should be: +# `platform = linux2|darwin|...` +# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes +platform = linux +commands = + # TODO(BEAM-8954): Remove this build_ext invocation once local source no longer + # shadows the installed apache_beam. + python setup.py build_ext --inplace + python apache_beam/examples/complete/autocomplete_test.py + {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + [testenv:py27-cloud] extras = test,gcp,aws commands = @@ -149,6 +167,11 @@ extras = test,gcp,interactive,aws commands = {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" +[testenv:py38-cloud] +extras = test,gcp,interactive,aws +commands = + {toxinidir}/scripts/run_pytest.sh {envname} "{posargs}" + [testenv:py27-lint] # Checks for py2 syntax errors deps = @@ -197,7 +220,7 @@ commands = mypy --version python setup.py mypy -[testenv:py37-docs] +[testenv:py38-docs] extras = test,gcp,docs,interactive deps = Sphinx==3.0.3 diff --git a/settings.gradle b/settings.gradle index 8a80ffd06298..e7bceb4c5c03 100644 --- a/settings.gradle +++ b/settings.gradle @@ -155,6 +155,7 @@ include ":sdks:python:test-suites:tox:py2" include ":sdks:python:test-suites:tox:py35" include ":sdks:python:test-suites:tox:py36" include ":sdks:python:test-suites:tox:py37" +include ":sdks:python:test-suites:tox:py38" include ":vendor:grpc-1_26_0" include ":vendor:bytebuddy-1_10_8" include ":vendor:calcite-1_20_0" From c0c786d41e36d1d1359c1fb96046d24f2a0326f3 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Thu, 14 May 2020 18:14:49 +0200 Subject: [PATCH 083/151] [BEAM-9810] Fix 'dictionary keys changed during iteration' error --- .../apache_beam/io/hadoopfilesystem_test.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/io/hadoopfilesystem_test.py b/sdks/python/apache_beam/io/hadoopfilesystem_test.py index a4b513b5876b..e83091f145c9 100644 --- a/sdks/python/apache_beam/io/hadoopfilesystem_test.py +++ b/sdks/python/apache_beam/io/hadoopfilesystem_test.py @@ -190,12 +190,15 @@ def rename(self, path1, path2): if self.status(path1, strict=False) is None: raise FakeHdfsError('Path1 not found: %s' % path1) - for fullpath in self.files.keys(): # pylint: disable=consider-iterating-dictionary - if fullpath == path1 or fullpath.startswith(path1 + '/'): - f = self.files.pop(fullpath) - newpath = path2 + fullpath[len(path1):] - f.stat['path'] = newpath - self.files[newpath] = f + files_to_rename = [ + path for path in self.files + if path == path1 or path.startswith(path1 + '/') + ] + for fullpath in files_to_rename: + f = self.files.pop(fullpath) + newpath = path2 + fullpath[len(path1):] + f.stat['path'] = newpath + self.files[newpath] = f def checksum(self, path): f = self.files.get(path, None) From c2b9867d9374dde9cc93c65d694e63acf276e4bb Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Wed, 13 May 2020 16:00:27 +0200 Subject: [PATCH 084/151] [BEAM-9810] Disable all failing tests until Dataflow runner supports Python 3.8 --- .../runners/dataflow/dataflow_runner_test.py | 57 ++++++++++++++++ .../dataflow/internal/apiclient_test.py | 66 +++++++++++++++++++ .../runners/dataflow/template_runner_test.py | 3 + 3 files changed, 126 insertions(+) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 32ace1a8eef8..16ea3362661a 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -225,6 +225,9 @@ def test_environment_override_translation(self): capabilities=environments.python_sdk_capabilities()) ]) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_remote_runner_translation(self): remote_runner = DataflowRunner() with Pipeline(remote_runner, @@ -235,6 +238,9 @@ def test_remote_runner_translation(self): | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)]) | ptransform.GroupByKey()) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_streaming_create_translation(self): remote_runner = DataflowRunner() self.default_properties.append("--streaming") @@ -250,6 +256,9 @@ def test_streaming_create_translation(self): self.assertEqual(job_dict[u'steps'][1][u'kind'], u'ParallelDo') self.assertEqual(job_dict[u'steps'][2][u'kind'], u'ParallelDo') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_bigquery_read_streaming_fail(self): remote_runner = DataflowRunner() self.default_properties.append("--streaming") @@ -259,6 +268,9 @@ def test_bigquery_read_streaming_fail(self): PipelineOptions(self.default_properties)) as p: _ = p | beam.io.Read(beam.io.BigQuerySource('some.table')) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_biqquery_read_fn_api_fail(self): remote_runner = DataflowRunner() for flag in ['beam_fn_api', 'use_unified_worker', 'use_runner_v2']: @@ -271,6 +283,9 @@ def test_biqquery_read_fn_api_fail(self): PipelineOptions(self.default_properties)) as p: _ = p | beam.io.Read(beam.io.BigQuerySource('some.table')) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_remote_runner_display_data(self): remote_runner = DataflowRunner() p = Pipeline( @@ -313,6 +328,9 @@ def test_remote_runner_display_data(self): }] self.assertUnhashableCountEqual(disp_data, expected_data) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_no_group_by_key_directly_after_bigquery(self): remote_runner = DataflowRunner() with self.assertRaises(ValueError, @@ -438,6 +456,9 @@ def test_side_input_visitor(self): common_urns.side_inputs.MULTIMAP.urn, side_input._side_input_data().access_pattern) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_min_cpu_platform_flag_is_propagated_to_experiments(self): remote_runner = DataflowRunner() self.default_properties.append('--min_cpu_platform=Intel Haswell') @@ -448,6 +469,9 @@ def test_min_cpu_platform_flag_is_propagated_to_experiments(self): 'min_cpu_platform=Intel Haswell', remote_runner.job.options.view_as(DebugOptions).experiments) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_streaming_engine_flag_adds_windmill_experiments(self): remote_runner = DataflowRunner() self.default_properties.append('--streaming') @@ -463,6 +487,9 @@ def test_streaming_engine_flag_adds_windmill_experiments(self): self.assertIn('enable_windmill_service', experiments_for_job) self.assertIn('some_other_experiment', experiments_for_job) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_upload_graph_experiment(self): remote_runner = DataflowRunner() self.default_properties.append('--experiment=upload_graph') @@ -474,6 +501,9 @@ def test_upload_graph_experiment(self): remote_runner.job.options.view_as(DebugOptions).experiments) self.assertIn('upload_graph', experiments_for_job) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_dataflow_worker_jar_flag_non_fnapi_noop(self): remote_runner = DataflowRunner() self.default_properties.append('--experiment=some_other_experiment') @@ -487,6 +517,9 @@ def test_dataflow_worker_jar_flag_non_fnapi_noop(self): self.assertIn('some_other_experiment', experiments_for_job) self.assertNotIn('use_staged_dataflow_worker_jar', experiments_for_job) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_dataflow_worker_jar_flag_adds_use_staged_worker_jar_experiment(self): remote_runner = DataflowRunner() self.default_properties.append('--experiment=beam_fn_api') @@ -500,6 +533,9 @@ def test_dataflow_worker_jar_flag_adds_use_staged_worker_jar_experiment(self): self.assertIn('beam_fn_api', experiments_for_job) self.assertIn('use_staged_dataflow_worker_jar', experiments_for_job) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_use_fastavro_experiment_is_added_on_py3_and_onwards(self): remote_runner = DataflowRunner() @@ -511,6 +547,9 @@ def test_use_fastavro_experiment_is_added_on_py3_and_onwards(self): remote_runner.job.options.view_as(DebugOptions).lookup_experiment( 'use_fastavro', False)) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_use_fastavro_experiment_is_not_added_when_use_avro_is_present(self): remote_runner = DataflowRunner() self.default_properties.append('--experiment=use_avro') @@ -522,6 +561,9 @@ def test_use_fastavro_experiment_is_not_added_when_use_avro_is_present(self): self.assertFalse(debug_options.lookup_experiment('use_fastavro', False)) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_unsupported_fnapi_features(self): remote_runner = DataflowRunner() self.default_properties.append('--experiment=beam_fn_api') @@ -575,6 +617,9 @@ def test_get_default_gcp_region_ignores_error( result = runner.get_default_gcp_region() self.assertIsNone(result) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_combine_values_translation(self): runner = DataflowRunner() @@ -625,6 +670,9 @@ def expect_correct_override(self, job, step_name, step_kind): self.assertGreater(len(step[u'properties']['display_data']), 0) self.assertEqual(step[u'properties']['output_info'], expected_output_info) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_read_create_translation(self): runner = DataflowRunner() @@ -635,6 +683,9 @@ def test_read_create_translation(self): self.expect_correct_override(runner.job, u'Create/Read', u'ParallelRead') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_read_bigquery_translation(self): runner = DataflowRunner() @@ -645,6 +696,9 @@ def test_read_bigquery_translation(self): self.expect_correct_override(runner.job, u'Read', u'ParallelRead') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_read_pubsub_translation(self): runner = DataflowRunner() @@ -658,6 +712,9 @@ def test_read_pubsub_translation(self): self.expect_correct_override( runner.job, u'ReadFromPubSub/Read', u'ParallelRead') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_gbk_translation(self): runner = DataflowRunner() with beam.Pipeline(runner=runner, diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py index 7776156bd460..b42ab12d0e35 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py @@ -61,6 +61,9 @@ def test_create_application_client(self): pipeline_options = PipelineOptions() apiclient.DataflowApplicationClient(pipeline_options) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_pipeline_url(self): pipeline_options = PipelineOptions([ '--subnetwork', @@ -92,6 +95,9 @@ def test_pipeline_url(self): self.assertEqual(pipeline_url.string_value, FAKE_PIPELINE_URL) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_set_network(self): pipeline_options = PipelineOptions([ '--network', @@ -105,6 +111,9 @@ def test_set_network(self): FAKE_PIPELINE_URL) self.assertEqual(env.proto.workerPools[0].network, 'anetworkname') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_set_subnetwork(self): pipeline_options = PipelineOptions([ '--subnetwork', @@ -121,6 +130,9 @@ def test_set_subnetwork(self): env.proto.workerPools[0].subnetwork, '/regions/MY/subnetworks/SUBNETWORK') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_flexrs_blank(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp']) @@ -131,6 +143,9 @@ def test_flexrs_blank(self): FAKE_PIPELINE_URL) self.assertEqual(env.proto.flexResourceSchedulingGoal, None) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_flexrs_cost(self): pipeline_options = PipelineOptions([ '--flexrs_goal', @@ -149,6 +164,9 @@ def test_flexrs_cost(self): dataflow.Environment.FlexResourceSchedulingGoalValueValuesEnum. FLEXRS_COST_OPTIMIZED)) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_flexrs_speed(self): pipeline_options = PipelineOptions([ '--flexrs_goal', @@ -167,6 +185,9 @@ def test_flexrs_speed(self): dataflow.Environment.FlexResourceSchedulingGoalValueValuesEnum. FLEXRS_SPEED_OPTIMIZED)) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_sdk_harness_container_images_get_set(self): pipeline_options = PipelineOptions([ @@ -350,6 +371,9 @@ def test_translate_means_using_distribution_accumulator(self): self.assertEqual( metric_update.floatingPointMean.count.lowBits, accumulator.count) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_default_ip_configuration(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp']) @@ -359,6 +383,9 @@ def test_default_ip_configuration(self): FAKE_PIPELINE_URL) self.assertEqual(env.proto.workerPools[0].ipConfiguration, None) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_public_ip_configuration(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp', '--use_public_ips']) @@ -370,6 +397,9 @@ def test_public_ip_configuration(self): env.proto.workerPools[0].ipConfiguration, dataflow.WorkerPool.IpConfigurationValueValuesEnum.WORKER_IP_PUBLIC) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_private_ip_configuration(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp', '--no_use_public_ips']) @@ -381,6 +411,9 @@ def test_private_ip_configuration(self): env.proto.workerPools[0].ipConfiguration, dataflow.WorkerPool.IpConfigurationValueValuesEnum.WORKER_IP_PRIVATE) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_number_of_worker_harness_threads(self): pipeline_options = PipelineOptions([ '--temp_location', @@ -398,6 +431,9 @@ def test_number_of_worker_harness_threads(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_default_in_released_sdks(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp', '--streaming']) @@ -416,6 +452,9 @@ def test_harness_override_default_in_released_sdks(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_absent_in_released_sdks_with_runner_v2(self): pipeline_options = PipelineOptions([ '--temp_location', @@ -436,6 +475,9 @@ def test_harness_override_absent_in_released_sdks_with_runner_v2(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_custom_in_released_sdks(self): pipeline_options = PipelineOptions([ '--temp_location', @@ -460,6 +502,9 @@ def test_harness_override_custom_in_released_sdks(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_custom_in_released_sdks_with_runner_v2(self): pipeline_options = PipelineOptions([ '--temp_location', @@ -485,6 +530,9 @@ def test_harness_override_custom_in_released_sdks_with_runner_v2(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0.rc1') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_uses_base_version_in_rc_releases(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp', '--streaming']) @@ -503,6 +551,9 @@ def test_harness_override_uses_base_version_in_rc_releases(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0.dev') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_harness_override_absent_in_unreleased_sdk(self): pipeline_options = PipelineOptions( ['--temp_location', 'gs://any-location/temp', '--streaming']) @@ -518,6 +569,9 @@ def test_harness_override_absent_in_unreleased_sdk(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0.dev') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_pinned_worker_harness_image_tag_used_in_dev_sdk(self): # streaming, fnapi pipeline. pipeline_options = PipelineOptions( @@ -580,6 +634,9 @@ def test_pinned_worker_harness_image_tag_used_in_dev_sdk(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0') + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_worker_harness_image_tag_matches_released_sdk_version(self): # streaming, fnapi pipeline. pipeline_options = PipelineOptions( @@ -630,6 +687,9 @@ def test_worker_harness_image_tag_matches_released_sdk_version(self): 'apache_beam.runners.dataflow.internal.apiclient.' 'beam_version.__version__', '2.2.0.rc1') + @unittest.skipIf( + sys.version_info.minor == 8, 'Re-enable once BEAM-9754 is ' + 'resolved') def test_worker_harness_image_tag_matches_base_sdk_version_of_an_rc(self): # streaming, fnapi pipeline. pipeline_options = PipelineOptions( @@ -676,6 +736,9 @@ def test_worker_harness_image_tag_matches_base_sdk_version_of_an_rc(self): names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:2.2.0' % (sys.version_info[0], sys.version_info[1]))) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_worker_harness_override_takes_precedence_over_sdk_defaults(self): # streaming, fnapi pipeline. pipeline_options = PipelineOptions([ @@ -765,6 +828,9 @@ def test_labels(self): self.assertEqual('key5', job.proto.labels.additionalProperties[4].key) self.assertEqual('', job.proto.labels.additionalProperties[4].value) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_experiment_use_multiple_sdk_containers(self): pipeline_options = PipelineOptions([ '--project', diff --git a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py index 54e39a5ee21a..3c173924f006 100644 --- a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py @@ -22,6 +22,7 @@ from __future__ import absolute_import import json +import sys import tempfile import unittest @@ -40,6 +41,8 @@ @unittest.skipIf(apiclient is None, 'GCP dependencies are not installed') +@unittest.skipIf( + sys.version_info.minor == 8, 'Doesn\'t work on Python 3.8, see: BEAM-9754') class TemplatingDataflowRunnerTest(unittest.TestCase): """TemplatingDataflow tests.""" def test_full_completion(self): From 63b4d3a27a5e6400cf8cf812bc46ed5350f36bde Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 9 Apr 2020 02:06:21 +0200 Subject: [PATCH 085/151] [BEAM-7770] Make ReplicaInfo part of Read on SolrIO --- .../org/apache/beam/sdk/io/solr/SolrIO.java | 47 ++++++++++++------- 1 file changed, 29 insertions(+), 18 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 747ac86c881b..01fc612d1b52 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -41,7 +41,6 @@ import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Sleeper; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -306,17 +305,22 @@ public abstract static class Read extends PTransform 0 && batchSize < MAX_BATCH_SIZE, - "Valid values for batchSize are 1 (inclusize) to %s (exclusive), but was: %s ", + "Valid values for batchSize are 1 (inclusive) to %s (exclusive), but was: %s ", MAX_BATCH_SIZE, batchSize); return builder().setBatchSize(batchSize).build(); } + /** Read from a specific Replica (partition). */ + public Read withReplicaInfo(ReplicaInfo replicaInfo) { + checkArgument(replicaInfo != null, "replicaInfo can not be null"); + return builder().setReplicaInfo(replicaInfo).build(); + } + @Override public PCollection expand(PBegin input) { checkArgument( @@ -386,14 +396,18 @@ public PCollection expand(PBegin input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.addIfNotNull(DisplayData.item("query", getQuery())); getConnectionConfiguration().populateDisplayData(builder); + builder.add(DisplayData.item("collection", getCollection())); + builder.addIfNotNull(DisplayData.item("query", getQuery())); + builder.add(DisplayData.item("batchSize", getBatchSize())); + final String replicaInfo = (getReplicaInfo() != null) ? getReplicaInfo().toString() : null; + builder.addIfNotNull(DisplayData.item("replicaInfo", replicaInfo)); } } /** A POJO describing a replica of Solr. */ @AutoValue - abstract static class ReplicaInfo implements Serializable { + public abstract static class ReplicaInfo implements Serializable { public abstract String coreName(); public abstract String coreUrl(); @@ -408,10 +422,9 @@ static ReplicaInfo create(Replica replica) { } } - static class SplitFn extends DoFn> { + private static class SplitFn extends DoFn { @ProcessElement - public void process(@Element SolrIO.Read spec, OutputReceiver> out) - throws IOException { + public void process(@Element Read spec, OutputReceiver out) throws IOException { ConnectionConfiguration connectionConfig = spec.getConnectionConfiguration(); try (AuthorizedSolrClient client = connectionConfig.createClient()) { String collection = spec.getCollection(); @@ -437,19 +450,17 @@ public void process(@Element SolrIO.Read spec, OutputReceiver, SolrDocument> { + private static class ReadFn extends DoFn { @ProcessElement - public void process( - @Element KV specAndReplica, OutputReceiver out) - throws IOException { - Read spec = specAndReplica.getKey(); - ReplicaInfo replica = specAndReplica.getValue(); + public void process(@Element Read spec, OutputReceiver out) throws IOException { + ReplicaInfo replicaInfo = spec.getReplicaInfo(); + checkArgument(replicaInfo != null, "replicaInfo is required"); String cursorMark = CursorMarkParams.CURSOR_MARK_START; String query = spec.getQuery(); if (query == null) { @@ -459,7 +470,7 @@ public void process( solrQuery.setRows(spec.getBatchSize()); solrQuery.setDistrib(false); try (AuthorizedSolrClient client = - spec.getConnectionConfiguration().createClient(replica.baseUrl())) { + spec.getConnectionConfiguration().createClient(replicaInfo.baseUrl())) { SchemaRequest.UniqueKey request = new SchemaRequest.UniqueKey(); try { SchemaResponse.UniqueKeyResponse response = client.process(spec.getCollection(), request); @@ -472,7 +483,7 @@ public void process( solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark); try { QueryResponse response; - response = client.query(replica.coreName(), solrQuery); + response = client.query(replicaInfo.coreName(), solrQuery); if (cursorMark.equals(response.getNextCursorMark())) { break; } @@ -599,7 +610,7 @@ static class WriteFn extends DoFn { } @Setup - public void setup() throws Exception { + public void setup() { solrClient = spec.getConnectionConfiguration().createClient(); retryBackoff = From 61eebf0e4623e67ef9f5430052ea5d26c7b69d0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 9 Apr 2020 02:33:46 +0200 Subject: [PATCH 086/151] [BEAM-7770] Add ReadAll transform for SolrIO --- .../org/apache/beam/sdk/io/solr/SolrIO.java | 21 +++++++++++++------ .../apache/beam/sdk/io/solr/SolrIOTest.java | 17 +++++++++++++++ 2 files changed, 32 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 01fc612d1b52..7b23839ce083 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -123,6 +123,10 @@ public static Read read() { return new AutoValue_SolrIO_Read.Builder().setBatchSize(1000).setQuery("*:*").build(); } + public static ReadAll readAll() { + return new ReadAll(); + } + public static Write write() { // 1000 for batch size is good enough in many cases, // ex: if document size is large, around 10KB, the request's size will be around 10MB @@ -385,12 +389,7 @@ public PCollection expand(PBegin input) { checkArgument( getConnectionConfiguration() != null, "withConnectionConfiguration() is required"); checkArgument(getCollection() != null, "from() is required"); - - return input - .apply("Create", Create.of(this)) - .apply("Split", ParDo.of(new SplitFn())) - .apply("Reshuffle", Reshuffle.viaRandomKey()) - .apply("Read", ParDo.of(new ReadFn())); + return input.apply("Create", Create.of(this)).apply("ReadAll", readAll()); } @Override @@ -499,6 +498,16 @@ public void process(@Element Read spec, OutputReceiver out) throws } } + public static class ReadAll extends PTransform, PCollection> { + @Override + public PCollection expand(PCollection input) { + return input + .apply("Split", ParDo.of(new SplitFn())) + .apply("Reshuffle", Reshuffle.viaRandomKey()) + .apply("Read", ParDo.of(new ReadFn())); + } + } + /** A {@link PTransform} writing data to Solr. */ @AutoValue public abstract static class Write extends PTransform, PDone> { diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 09c38a2b57e6..6f666b8600db 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -155,6 +155,23 @@ public void testRead() throws Exception { pipeline.run(); } + @Test + public void testReadAll() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PCollection output = + pipeline + .apply( + Create.of( + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .from(SOLR_COLLECTION) + .withBatchSize(101))) + .apply(SolrIO.readAll()); + PAssert.thatSingleton(output.apply("Count", Count.globally())).isEqualTo(NUM_DOCS); + pipeline.run(); + } + @Test public void testReadWithQuery() throws Exception { SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); From 9199598ea6bc185822fc888fa6dc376dacb9d641 Mon Sep 17 00:00:00 2001 From: Piotr Szuberski Date: Wed, 27 May 2020 15:41:50 +0200 Subject: [PATCH 087/151] [BEAM-10102] Fix query in pubsub read grafana dashoard --- .../perftests_metrics/Python_IO_IT_Tests_Dataflow.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json b/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json index d11465bc14fb..c7bf2e390862 100644 --- a/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json +++ b/.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json @@ -319,7 +319,7 @@ "measurement": "python_bqio_read", "orderByTime": "ASC", "policy": "default", - "query": "SELECT mean(\"value\") FROM \"python_psio_2GB_results\" WHERE \"metric\" = \"pubsub_io_perf_read_runtime\" AND $timeFilter GROUP BY time($__interval), \"metric\"", + "query": "SELECT mean(\"value\") FROM \"python_psio_2GB_results\" WHERE \"metric\" = 'pubsub_io_perf_read_runtime' AND $timeFilter GROUP BY time($__interval), \"metric\"", "rawQuery": true, "refId": "A", "resultFormat": "time_series", From 4676c438b087b1ddbdb49547fddae6b5edcb1e0c Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 10:13:40 -0400 Subject: [PATCH 088/151] [BEAM-10106] Script the deployment of artifacts to pypi --- release/src/main/scripts/deploy_pypi.sh | 57 +++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100755 release/src/main/scripts/deploy_pypi.sh diff --git a/release/src/main/scripts/deploy_pypi.sh b/release/src/main/scripts/deploy_pypi.sh new file mode 100755 index 000000000000..c6b3dcd5056f --- /dev/null +++ b/release/src/main/scripts/deploy_pypi.sh @@ -0,0 +1,57 @@ +#!/bin/bash +# +# 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. +# + +# This script uploads Python artifacts staged at dist.apache.org to PyPI. + +set -e + +function clean_up(){ + echo "Do you want to clean local clone repo? [y|N]" + read confirmation + if [[ $confirmation = "y" ]]; then + cd ~ + rm -rf ${LOCAL_CLONE_DIR} + echo "Clean up local repo." + fi +} + +echo "Enter the release version, e.g. 2.21.0:" +read RELEASE +LOCAL_CLONE_DIR="beam_release_${RELEASE}" +cd ~ +if [[ -d ${LOCAL_CLONE_DIR} ]]; then + rm -rf ${LOCAL_CLONE_DIR} +fi +mkdir ${LOCAL_CLONE_DIR} +cd ${LOCAL_CLONE_DIR} + +wget -r --no-parent -A zip,whl "https://dist.apache.org/repos/dist/dev/beam/${RELEASE}/python" +cd "dist.apache.org/repos/dist/dev/beam/${RELEASE}/python/" +pip install twine +echo "Will upload the following files to PyPI:" +ls +echo "Are the files listed correct? [y|N]" +read confirmation +if [[ $confirmation != "y" ]]; then + echo "Exit without deploying artifacts to PyPI." + clean_up + exit +fi +twine upload * + +clean_up From 10732c7549a09015b97f18e80b2af69b0c8a4390 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 10:41:09 -0400 Subject: [PATCH 089/151] Add pypi script to release guide. --- .../site/content/en/contribute/release-guide.md | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 7e199ae9d5ab..37e5b6bb67a6 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -1271,13 +1271,14 @@ please follow [the guide](https://help.github.com/articles/creating-a-personal-a ### Deploy Python artifacts to PyPI -1. Download everything from https://dist.apache.org/repos/dist/dev/beam/2.14.0/python/ ; -2. Keep only things that you see in https://pypi.org/project/apache-beam/#files , e.g. `.zip`, `.whl`, - delete the `.asc`, `.sha512`; -3. Upload the new release `twine upload *` from the directory with the `.zip` and `.whl` files; - -[Installing twine](https://packaging.python.org/tutorials/packaging-projects/#uploading-the-distribution-archives): `pip install twine`. You can install twine under [virtualenv](https://virtualenv.pypa.io/en/latest/) if preferred. - +* Script: [deploy_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_pypi.sh) +* Usage +``` +./beam/release/src/main/scripts/deploy_pypi.sh +``` +* Verify that the files at https://pypi.org/project/apache-beam/#files are correct. +All wheels should be published, in addition to the zip of the release source. +(Signatures and hashes do _not_ need to be uploaded.) ### Deploy source release to dist.apache.org From ec7278b935769b2ebed4b2ef27b55169cea28551 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 08:15:11 -0700 Subject: [PATCH 090/151] [BEAM-10108] Update Flink versions in publish_docker_images.sh. --- release/src/main/scripts/publish_docker_images.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release/src/main/scripts/publish_docker_images.sh b/release/src/main/scripts/publish_docker_images.sh index 1fb646313861..7575107ebf11 100755 --- a/release/src/main/scripts/publish_docker_images.sh +++ b/release/src/main/scripts/publish_docker_images.sh @@ -28,7 +28,7 @@ DOCKER_IMAGE_DEFAULT_REPO_ROOT=apache DOCKER_IMAGE_DEFAULT_REPO_PREFIX=beam_ PYTHON_VER=("python2.7" "python3.5" "python3.6" "python3.7") -FLINK_VER=("1.7" "1.8" "1.9") +FLINK_VER=("1.8" "1.9" "1.10") echo "Publish SDK docker images to Docker Hub." From 5129587b16064c9b11903de895306ce630a4ddbb Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 11:34:14 -0400 Subject: [PATCH 091/151] Add release date. --- website/www/site/content/en/get-started/downloads.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 14ae4d683162..a9575e4c73ff 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -87,7 +87,7 @@ versions denoted `0.x.y`. ## Releases -### 2.21.0 (2020-05-xx) +### 2.21.0 (2020-05-27) Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.21.0/apache-beam-2.21.0-source-release.zip). [SHA-512](https://downloads.apache.org/beam/2.21.0/apache-beam-2.21.0-source-release.zip.sha512). [signature](https://downloads.apache.org/beam/2.21.0/apache-beam-2.21.0-source-release.zip.asc). @@ -248,4 +248,3 @@ Official [source code download](https://archive.apache.org/dist/beam/2.0.0/apach [signature](https://archive.apache.org/dist/beam/2.0.0/apache-beam-2.0.0-source-release.zip.asc). [Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12339746). - From ff8430ea42b51a35e7316be6cb1c14150dca0afc Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 11:35:59 -0400 Subject: [PATCH 092/151] Add release date. --- CHANGES.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index c55967a18bfb..1191ab8d2f35 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -75,7 +75,7 @@ * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). -# [2.21.0] - 2020-05-xx +# [2.21.0] - 2020-05-27 ## Highlights From c5b11834414b6bce01bef05659d5d0770368ee2b Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 11:36:11 -0400 Subject: [PATCH 093/151] Add release date. --- website/www/site/content/en/blog/beam-2.21.0.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/beam-2.21.0.md b/website/www/site/content/en/blog/beam-2.21.0.md index 286d71b6a20b..d46f432e3d7f 100644 --- a/website/www/site/content/en/blog/beam-2.21.0.md +++ b/website/www/site/content/en/blog/beam-2.21.0.md @@ -1,6 +1,6 @@ --- title: "Apache Beam 2.21.0" -date: 2020-05-xx 00:00:01 -0800 +date: 2020-05-27 00:00:01 -0800 categories: - blog authors: From e0f858f852a8b7ab7ede6c0d5e55d4653ea59ccd Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 11:36:29 -0400 Subject: [PATCH 094/151] Remove HTML comment (causes formatting issues) --- website/www/site/content/en/blog/beam-2.21.0.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/beam-2.21.0.md b/website/www/site/content/en/blog/beam-2.21.0.md index d46f432e3d7f..734642148d48 100644 --- a/website/www/site/content/en/blog/beam-2.21.0.md +++ b/website/www/site/content/en/blog/beam-2.21.0.md @@ -19,7 +19,7 @@ limitations under the License. --> We are happy to present the new 2.21.0 release of Beam. This release includes both improvements and new functionality. -See the [download page](/get-started/downloads/#xxxx-xxxx) for this release. +See the [download page](/get-started/downloads/#xxxx-xxxx) for this release. For more information on changes in 2.21.0, check out the [detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347143). From 5b6e84fa380f1586bdd7ec1e8200e43ecc827f6e Mon Sep 17 00:00:00 2001 From: Steven Niemitz Date: Wed, 27 May 2020 11:52:52 -0400 Subject: [PATCH 095/151] Add a flag to tune the size of the grouping tables used by the dataflow workers (#11811) --- .../dataflow/worker/PartialGroupByKeyParDoFns.java | 10 ++++++++-- .../worker/util/common/worker/GroupingTables.java | 10 ++++++---- .../apache/beam/sdk/options/SdkHarnessOptions.java | 14 ++++++++++++++ .../beam/fn/harness/PrecombineGroupingTable.java | 13 +++++++------ 4 files changed, 35 insertions(+), 12 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index ffb617053ce1..86df406613de 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -39,6 +39,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.SdkHarnessOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -97,6 +98,9 @@ static ParDoFn create( Receiver receiver, @Nullable StepContext stepContext) throws Exception { + long maxSizeBytes = + options.as(SdkHarnessOptions.class).getGroupingTableMaxSizeMb() * (1024L * 1024L); + Coder keyCoder = inputElementCoder.getKeyCoder(); Coder valueCoder = inputElementCoder.getValueCoder(); if (combineFn == null) { @@ -108,7 +112,8 @@ static ParDoFn create( PairInfo.create(), new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(inputCoder), - 0.001 /*sizeEstimatorSampleRate*/); + 0.001, /*sizeEstimatorSampleRate*/ + maxSizeBytes /*maxSizeBytes*/); return new SimplePartialGroupByKeyParDoFn<>(groupingTable, receiver); } else { GroupingTables.Combiner, InputT, AccumT, ?> valueCombiner = @@ -122,7 +127,8 @@ static ParDoFn create( valueCombiner, new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(combineFn.getAccumulatorCoder()), - 0.001 /*sizeEstimatorSampleRate*/); + 0.001, /*sizeEstimatorSampleRate*/ + maxSizeBytes /*maxSizeBytes*/); if (sideInputReader.isEmpty()) { return new SimplePartialGroupByKeyParDoFn<>(groupingTable, receiver); } else if (options.as(StreamingOptions.class).isStreaming()) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java index d1ab37856546..ddccbf75b188 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java @@ -59,9 +59,10 @@ public static GroupingTable> bufferingAndSampling( PairInfo pairInfo, SizeEstimator keySizer, SizeEstimator valueSizer, - double sizeEstimatorSampleRate) { + double sizeEstimatorSampleRate, + long maxSizeBytes) { return new BufferingGroupingTable<>( - DEFAULT_MAX_GROUPING_TABLE_BYTES, + maxSizeBytes, groupingKeyCreator, pairInfo, new SamplingSizeEstimator<>(keySizer, sizeEstimatorSampleRate, 1.0), @@ -94,9 +95,10 @@ public static GroupingTable combiningAndS Combiner combineFn, SizeEstimator keySizer, SizeEstimator accumulatorSizer, - double sizeEstimatorSampleRate) { + double sizeEstimatorSampleRate, + long maxSizeBytes) { return new CombiningGroupingTable<>( - DEFAULT_MAX_GROUPING_TABLE_BYTES, + maxSizeBytes, groupingKeyCreator, pairInfo, combineFn, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java index c5ad12a3b4c8..5f1508168d45 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java @@ -82,6 +82,20 @@ enum LogLevel { void setSdkHarnessLogLevelOverrides(SdkHarnessLogLevelOverrides value); + /** + * Size (in MB) of each grouping table used to pre-combine elements. If unset, defaults to 100 MB. + * + *

    CAUTION: If set too large, workers may run into OOM conditions more easily, each worker may + * have many grouping tables in-memory concurrently. + */ + @Description( + "The size (in MB) of the grouping tables used to pre-combine elements before " + + "shuffling. Larger values may reduce the amount of data shuffled.") + @Default.Integer(100) + int getGroupingTableMaxSizeMb(); + + void setGroupingTableMaxSizeMb(int value); + /** * Defines a log level override for a specific class, package, or name. * diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java index 48a55bf3e8ae..515f0644c2ef 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java @@ -27,6 +27,7 @@ import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.SdkHarnessOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; @@ -40,6 +41,10 @@ /** Static utility methods that provide {@link GroupingTable} implementations. */ public class PrecombineGroupingTable implements GroupingTable { + private static long getGroupingTableSizeBytes(PipelineOptions options) { + return options.as(SdkHarnessOptions.class).getGroupingTableMaxSizeMb() * 1024L * 1024L; + } + /** Returns a {@link GroupingTable} that combines inputs into a accumulator. */ public static GroupingTable, InputT, AccumT> combining( PipelineOptions options, @@ -50,7 +55,7 @@ public static GroupingTable, InputT, AccumT new ValueCombiner<>( GlobalCombineFnRunners.create(combineFn), NullSideInputReader.empty(), options); return new PrecombineGroupingTable<>( - DEFAULT_MAX_GROUPING_TABLE_BYTES, + getGroupingTableSizeBytes(options), new WindowingCoderGroupingKeyCreator<>(keyCoder), WindowedPairInfo.create(), valueCombiner, @@ -73,7 +78,7 @@ GroupingTable, InputT, AccumT> combiningAndSampling( new ValueCombiner<>( GlobalCombineFnRunners.create(combineFn), NullSideInputReader.empty(), options); return new PrecombineGroupingTable<>( - DEFAULT_MAX_GROUPING_TABLE_BYTES, + getGroupingTableSizeBytes(options), new WindowingCoderGroupingKeyCreator<>(keyCoder), WindowedPairInfo.create(), valueCombiner, @@ -255,10 +260,6 @@ public OutputT extract(WindowedValue windowedKey, AccumT accumulator) { } } - // By default, how many bytes we allow the grouping table to consume before - // it has to be flushed. - private static final long DEFAULT_MAX_GROUPING_TABLE_BYTES = 100_000_000L; - // How many bytes a word in the JVM has. private static final int BYTES_PER_JVM_WORD = getBytesPerJvmWord(); /** From 1ec86e737422c3a9b3f2ed7201397eded3524d96 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Wed, 27 May 2020 10:50:11 -0700 Subject: [PATCH 096/151] Add Pub/Sub Lite to in progress IOs --- website/www/site/content/en/documentation/io/built-in.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/website/www/site/content/en/documentation/io/built-in.md b/website/www/site/content/en/documentation/io/built-in.md index ba72908a885f..e40884e1e798 100644 --- a/website/www/site/content/en/documentation/io/built-in.md +++ b/website/www/site/content/en/documentation/io/built-in.md @@ -61,6 +61,10 @@ This table contains I/O transforms that are currently planned or in-progress. St Neo4jJava BEAM-1857 + + Pub/Sub LiteJava + BEAM-10114 + RestIOJava BEAM-1946 From 9939a8f99ea0b45035a0969bda48f62f7e17f874 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 16:32:14 -0400 Subject: [PATCH 097/151] Move 2.20 download addresses to archive. --- website/www/site/content/en/get-started/downloads.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 14ae4d683162..a1697abb6fa3 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -95,9 +95,9 @@ Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.21.0 [Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347143). ### 2.20.0 (2020-04-15) -Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.20.0/apache-beam-2.20.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.20.0/apache-beam-2.20.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.20.0/apache-beam-2.20.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/dist/beam/2.20.0/apache-beam-2.20.0-source-release.zip). +[SHA-512](https://archive.apache.org/dist/beam/2.20.0/apache-beam-2.20.0-source-release.zip.sha512). +[signature](https://archive.apache.org/dist/beam/2.20.0/apache-beam-2.20.0-source-release.zip.asc). [Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12346780). From 00736f34913fa94051e818b37f43a3f588539c3d Mon Sep 17 00:00:00 2001 From: Pablo Estrada Date: Wed, 27 May 2020 14:06:41 -0700 Subject: [PATCH 098/151] [BEAM-10098] Enabling javadoc export for RabbitMqIO and KuduIO --- sdks/java/io/kudu/build.gradle | 2 +- sdks/java/io/rabbitmq/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/kudu/build.gradle b/sdks/java/io/kudu/build.gradle index 0090e13336e0..cda962d4d6ee 100644 --- a/sdks/java/io/kudu/build.gradle +++ b/sdks/java/io/kudu/build.gradle @@ -17,7 +17,7 @@ */ plugins { id 'org.apache.beam.module' } -applyJavaNature(exportJavadoc: false, automaticModuleName: 'org.apache.beam.sdk.io.kudu') +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.kudu') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() diff --git a/sdks/java/io/rabbitmq/build.gradle b/sdks/java/io/rabbitmq/build.gradle index a5f1055cce95..83cf7c4591ab 100644 --- a/sdks/java/io/rabbitmq/build.gradle +++ b/sdks/java/io/rabbitmq/build.gradle @@ -17,7 +17,7 @@ */ plugins { id 'org.apache.beam.module' } -applyJavaNature(exportJavadoc: false, automaticModuleName: 'org.apache.beam.sdk.io.rabbitmq') +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.rabbitmq') description = "Apache Beam :: SDKs :: Java :: IO :: RabbitMQ" ext.summary = "IO to read and write to a RabbitMQ broker." From eae2113d43f206e263dd2e6bff3cb19b95f5b81e Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 17:26:02 -0400 Subject: [PATCH 099/151] Improve log messages in release scripts. --- release/src/main/scripts/cut_release_branch.sh | 9 +++++---- release/src/main/scripts/deploy_pypi.sh | 7 ++++--- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/release/src/main/scripts/cut_release_branch.sh b/release/src/main/scripts/cut_release_branch.sh index b3bdc832ea71..6d0f91379708 100755 --- a/release/src/main/scripts/cut_release_branch.sh +++ b/release/src/main/scripts/cut_release_branch.sh @@ -24,12 +24,12 @@ set -e function clean_up(){ - echo "Do you want to clean local clone repo? [y|N]" + echo "Do you want to clean local clone repo ${LOCAL_CLONE_DIR}? [y|N]" read confirmation if [[ $confirmation = "y" ]]; then cd ~ rm -rf ${LOCAL_CLONE_DIR} - echo "Clean up local repo." + echo "Cleaned up local repo." fi } @@ -73,6 +73,7 @@ echo "===============================================================" cd ~ if [[ -d ${LOCAL_CLONE_DIR} ]]; then + echo "Deleting existing local clone repo ${LOCAL_CLONE_DIR}." rm -rf ${LOCAL_CLONE_DIR} fi mkdir ${LOCAL_CLONE_DIR} @@ -101,7 +102,7 @@ echo "===============================================================" echo "Please make sure all changes above are expected. Do you confirm to commit?: [y|N]" read confirmation if [[ $confirmation != "y" ]]; then - echo "Exit without committing any changes on master branch." + echo "Exiting without committing any changes on master branch." clean_up exit fi @@ -136,7 +137,7 @@ echo "===============================================================" echo "Please make sure all changes above are expected. Do you confirm to commit?: [y|N]" read confirmation if [[ $confirmation != "y" ]]; then - echo "Exit without committing any changes on release branch." + echo "Exiting without committing any changes on release branch." clean_up exit fi diff --git a/release/src/main/scripts/deploy_pypi.sh b/release/src/main/scripts/deploy_pypi.sh index c6b3dcd5056f..5be84aa596e7 100755 --- a/release/src/main/scripts/deploy_pypi.sh +++ b/release/src/main/scripts/deploy_pypi.sh @@ -21,12 +21,12 @@ set -e function clean_up(){ - echo "Do you want to clean local clone repo? [y|N]" + echo "Do you want to clean local clone repo ${LOCAL_CLONE_DIR}? [y|N]" read confirmation if [[ $confirmation = "y" ]]; then cd ~ rm -rf ${LOCAL_CLONE_DIR} - echo "Clean up local repo." + echo "Cleaned up local repo." fi } @@ -35,6 +35,7 @@ read RELEASE LOCAL_CLONE_DIR="beam_release_${RELEASE}" cd ~ if [[ -d ${LOCAL_CLONE_DIR} ]]; then + echo "Deleting existing local clone repo ${LOCAL_CLONE_DIR}." rm -rf ${LOCAL_CLONE_DIR} fi mkdir ${LOCAL_CLONE_DIR} @@ -48,7 +49,7 @@ ls echo "Are the files listed correct? [y|N]" read confirmation if [[ $confirmation != "y" ]]; then - echo "Exit without deploying artifacts to PyPI." + echo "Exiting without deploying artifacts to PyPI." clean_up exit fi From b6100fd017c43f6a9e10307574ece718a152943b Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 17:30:37 -0400 Subject: [PATCH 100/151] Install twine in virtual env. --- release/src/main/scripts/deploy_pypi.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/release/src/main/scripts/deploy_pypi.sh b/release/src/main/scripts/deploy_pypi.sh index 5be84aa596e7..daeb342a621e 100755 --- a/release/src/main/scripts/deploy_pypi.sh +++ b/release/src/main/scripts/deploy_pypi.sh @@ -41,9 +41,12 @@ fi mkdir ${LOCAL_CLONE_DIR} cd ${LOCAL_CLONE_DIR} +virtualenv deploy_pypi_env +source ./deploy_pypi_env/bin/activate +pip install twine + wget -r --no-parent -A zip,whl "https://dist.apache.org/repos/dist/dev/beam/${RELEASE}/python" cd "dist.apache.org/repos/dist/dev/beam/${RELEASE}/python/" -pip install twine echo "Will upload the following files to PyPI:" ls echo "Are the files listed correct? [y|N]" From 0e7447419e5568081601156a998f0151aeec7891 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 27 May 2020 14:52:21 -0700 Subject: [PATCH 101/151] remove as_cloud_object override --- sdks/python/apache_beam/coders/row_coder.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py index 7012a56cb374..03114976c78e 100644 --- a/sdks/python/apache_beam/coders/row_coder.py +++ b/sdks/python/apache_beam/coders/row_coder.py @@ -67,9 +67,6 @@ def is_deterministic(self): def to_type_hint(self): return named_tuple_from_schema(self.schema) - def as_cloud_object(self, coders_context=None): - raise NotImplementedError("as_cloud_object not supported for RowCoder") - def __hash__(self): return hash(self.schema.SerializeToString()) From eb59a848ce519e8186750f95b2cda02bc52c683b Mon Sep 17 00:00:00 2001 From: Kamil Gabryjelski Date: Thu, 28 May 2020 00:11:56 +0200 Subject: [PATCH 102/151] Merge pull request #11796 from [BEAM-10003] Use local code for building code samples on website * Use code samples from local files instead of fetching from github * Update markdown files to use code_sample shortcode * Change script name in gradle and package.json * Update documentation * Remove empty line * Synchronize with latest changes * Update comment for code_sample --- website/.gitignore | 2 +- website/CONTRIBUTE.md | 10 +- website/build.gradle | 10 +- ...ithub_samples.sh => build_code_samples.sh} | 12 ++- website/www/package.json | 2 +- .../io/built-in/google-bigquery.md | 50 +++++----- .../documentation/io/developing-io-python.md | 12 +-- .../en/documentation/patterns/bigqueryio.md | 4 +- .../documentation/patterns/custom-windows.md | 12 +-- .../documentation/patterns/file-processing.md | 10 +- .../patterns/pipeline-options.md | 6 +- .../en/documentation/patterns/side-inputs.md | 6 +- .../en/documentation/programming-guide.md | 94 +++++++++---------- .../documentation/sdks/python-type-safety.md | 22 ++--- .../transforms/python/elementwise/filter.md | 24 ++--- .../transforms/python/elementwise/flatmap.md | 36 +++---- .../transforms/python/elementwise/keys.md | 4 +- .../transforms/python/elementwise/kvswap.md | 4 +- .../transforms/python/elementwise/map.md | 32 +++---- .../transforms/python/elementwise/pardo.md | 12 +-- .../python/elementwise/partition.md | 12 +-- .../transforms/python/elementwise/regex.md | 36 +++---- .../transforms/python/elementwise/tostring.md | 12 +-- .../transforms/python/elementwise/values.md | 4 +- .../python/elementwise/withtimestamps.md | 16 ++-- .../en/get-started/mobile-gaming-example.md | 44 ++++----- .../en/get-started/wordcount-example.md | 24 ++--- .../site/layouts/shortcodes/code_sample.html | 23 +++++ .../layouts/shortcodes/github_sample.html | 15 --- 29 files changed, 280 insertions(+), 270 deletions(-) rename website/www/{build_github_samples.sh => build_code_samples.sh} (75%) create mode 100644 website/www/site/layouts/shortcodes/code_sample.html delete mode 100644 website/www/site/layouts/shortcodes/github_sample.html diff --git a/website/.gitignore b/website/.gitignore index 8191c0750dc0..bbed591c748a 100644 --- a/website/.gitignore +++ b/website/.gitignore @@ -16,7 +16,7 @@ content/ www/node_modules www/dist www/site/resources -www/site/github_samples +www/site/code_samples www/site/_config_branch_repo.toml www/yarn-error.log !www/site/content diff --git a/website/CONTRIBUTE.md b/website/CONTRIBUTE.md index 555fd2a3026e..5176af95dedf 100644 --- a/website/CONTRIBUTE.md +++ b/website/CONTRIBUTE.md @@ -66,7 +66,7 @@ www/ │ │ └── js │   └── themes │ └── docsy -├── build_github_samples.sh +├── build_code_samples.sh ├── check-links.sh # links checker └── package.json ``` @@ -252,20 +252,20 @@ A table markdown here. {{< /table >}} ``` -### Github sample +### Code sample -To retrieve a piece of code in github. +To retrieve a piece of code from Beam project. Usage: ``` -{{< github_sample "/path/to/file" selected_tag >}} +{{< code_sample "path/to/file" selected_tag >}} ``` Example: ``` -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/user_score.py" extract_and_sum_score >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/user_score.py" extract_and_sum_score >}} ``` ### Others diff --git a/website/build.gradle b/website/build.gradle index b5988d8e2b42..7f8e7f69342b 100644 --- a/website/build.gradle +++ b/website/build.gradle @@ -104,16 +104,16 @@ task installDependencies(type: Exec) { "${->startDockerContainer.containerId()}", 'yarn', 'install' } -// Run build_github_samples.sh to fetch github content -// which is used by github_sample shortcodes to inject snippets into codeblocks -task buildGithubSamples(type: Exec) { +// Run build_code_samples.sh to fetch Beam project content +// which is used by code_sample shortcodes to inject snippets into codeblocks +task buildCodeSamples(type: Exec) { commandLine 'docker', 'exec', '--workdir', "$dockerSourceDir", - "${->startDockerContainer.containerId()}", 'yarn', 'build_github_samples' + "${->startDockerContainer.containerId()}", 'yarn', 'build_code_samples' } task setupDockerContainer(type: Exec) { dependsOn startDockerContainer - finalizedBy initGitSubmodules, installDependencies, buildGithubSamples + finalizedBy initGitSubmodules, installDependencies, buildCodeSamples ext.containerId = { return startDockerContainer.containerId() } diff --git a/website/www/build_github_samples.sh b/website/www/build_code_samples.sh similarity index 75% rename from website/www/build_github_samples.sh rename to website/www/build_code_samples.sh index b47e7d1286cf..a42ba444324b 100755 --- a/website/www/build_github_samples.sh +++ b/website/www/build_code_samples.sh @@ -23,19 +23,21 @@ pushd "${MY_DIR}" &>/dev/null || exit 1 echo "Working directory: ${MY_DIR}" -DIST_DIR=${1:-"./site/github_samples"} +PROJECT_ROOT_REL_PATH=../.. + +DIST_DIR=${1:-"./site/code_samples"} echo "Dist directory: ${DIST_DIR}" CONTENT_DIR=${2:-"./site/content"} -mapfile -t github_urls < <(grep -rh "{{< github_sample" "${CONTENT_DIR}" | sed -e 's/^.*"\(.*\)".*$/\1/g' | sort | uniq | sed 's/\/blob\//\//g' | xargs -n 1 echo) +mapfile -t code_sample_uris < <(grep -rh "{{< code_sample" "${CONTENT_DIR}" | sed -e 's/^.*"\(.*\)".*$/\1/g' | sort | uniq | xargs -n 1 echo) mkdir -pv "${DIST_DIR}" -for url in "${github_urls[@]}" +for uri in "${code_sample_uris[@]}" do - fileName=$(echo "$url" | sed -e 's/\//_/g') - curl -o "$DIST_DIR"/"$fileName" "https://raw.githubusercontent.com$url" + fileName=$(echo "$uri" | sed -e 's/\//_/g') + cp "$PROJECT_ROOT_REL_PATH"/"$uri" "$DIST_DIR"/"$fileName" done popd &>/dev/null || exit 1 diff --git a/website/www/package.json b/website/www/package.json index a5b1cbea637d..3b3a2b2e4de7 100644 --- a/website/www/package.json +++ b/website/www/package.json @@ -5,7 +5,7 @@ "repository": "apache/beam", "license": "MIT", "scripts": { - "build_github_samples": "./build_github_samples.sh", + "build_code_samples": "./build_code_samples.sh", "develop": "cd site && hugo server", "build": "cross-env HUGO_ENV=production hugo -d ../dist -s site -v", "start": "hugo -d ../dist -s site -vw" diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md index d549f5569fed..16fa33e41819 100644 --- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md +++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md @@ -103,11 +103,11 @@ To specify a table with a string, use the format table name. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpec >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpec >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec >}} {{< /highlight >}} You can also omit `project_id` and use the `[dataset_id].[table_id]` format. If @@ -120,11 +120,11 @@ you omit the project ID, Beam uses the default project ID from your {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpecWithoutProject >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpecWithoutProject >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec_without_project >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec_without_project >}} {{< /highlight >}} #### Using a TableReference @@ -133,11 +133,11 @@ To specify a table with a `TableReference`, create a new `TableReference` using the three parts of the BigQuery table name. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpecObject >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTableSpecObject >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec_object >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_table_spec_object >}} {{< /highlight >}} @@ -173,11 +173,11 @@ shows the correct format for data types used when reading from and writing to BigQuery: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryDataTypes >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryDataTypes >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_data_types >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_data_types >}} {{< /highlight >}} @@ -276,11 +276,11 @@ The following code reads an entire table that contains weather station data and then extracts the `max_temperature` column. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadTable >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadTable >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_table >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_table >}} {{< /highlight >}} @@ -300,22 +300,22 @@ If you don't want to read an entire table, you can supply a query string to The following code uses a SQL query to only read the `max_temperature` column. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadQuery >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadQuery >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_query >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_query >}} {{< /highlight >}} You can also use BigQuery's standard SQL dialect with a query string, as shown in the following example: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadQueryStdSQL >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryReadQueryStdSQL >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_query_std_sql >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_read_query_std_sql >}} {{< /highlight >}} ### Using the BigQuery Storage API {#storage-api} @@ -546,11 +546,11 @@ The following example code shows how to create a `TableSchema` for a table with two fields (source and quote) of type string. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQuerySchemaObject >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQuerySchemaObject >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_schema_object >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_schema_object >}} {{< /highlight >}} @@ -591,11 +591,11 @@ The following example shows how to use a string to specify the same table schema as the previous example. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQuerySchemaJson >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQuerySchemaJson >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_schema >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_schema >}} {{< /highlight >}} @@ -680,11 +680,11 @@ collection. The following examples use this `PCollection` that contains quotes. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteInput >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteInput >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_write_input >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_write_input >}} {{< /highlight >}} @@ -704,11 +704,11 @@ creates a table if needed; if the table already exists, it will be replaced. {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteTable >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteTable >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_write >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_write >}} {{< /highlight >}} @@ -723,7 +723,7 @@ be replaced. {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteFunction >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteFunction >}} {{< /highlight >}} {{< paragraph class="language-java" >}} @@ -774,7 +774,7 @@ different table for each year. {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteDynamicDestinations >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryWriteDynamicDestinations >}} {{< /highlight >}} {{< highlight py >}} @@ -810,7 +810,7 @@ This example generates one partition per day. {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTimePartitioning >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryTimePartitioning >}} {{< /highlight >}} {{< highlight py >}} diff --git a/website/www/site/content/en/documentation/io/developing-io-python.md b/website/www/site/content/en/documentation/io/developing-io-python.md index 3e581e01dd13..47b3ed075714 100644 --- a/website/www/site/content/en/documentation/io/developing-io-python.md +++ b/website/www/site/content/en/documentation/io/developing-io-python.md @@ -183,13 +183,13 @@ See [AvroSource](https://github.com/apache/beam/blob/master/sdks/python/apache_b The following example, `CountingSource`, demonstrates an implementation of `BoundedSource` and uses the SDK-provided `RangeTracker` called `OffsetRangeTracker`. {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_new_source >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_new_source >}} {{< /highlight >}} To read data from the source in your pipeline, use the `Read` transform: {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_use_new_source >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_use_new_source >}} {{< /highlight >}} **Note:** When you create a source that end-users are going to use, we @@ -262,23 +262,23 @@ to `_CountingSource`. Then, create the wrapper `PTransform`, called `ReadFromCountingSource`: {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_new_ptransform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_new_ptransform >}} {{< /highlight >}} Finally, read from the source: {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_use_ptransform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_source_use_ptransform >}} {{< /highlight >}} For the sink, rename `SimpleKVSink` to `_SimpleKVSink`. Then, create the wrapper `PTransform`, called `WriteToKVSink`: {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_sink_new_ptransform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_sink_new_ptransform >}} {{< /highlight >}} Finally, write to the sink: {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_sink_use_ptransform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_custom_sink_use_ptransform >}} {{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/bigqueryio.md b/website/www/site/content/en/documentation/patterns/bigqueryio.md index 49393b873b41..1206c5deace6 100644 --- a/website/www/site/content/en/documentation/patterns/bigqueryio.md +++ b/website/www/site/content/en/documentation/patterns/bigqueryio.md @@ -38,9 +38,9 @@ In the result tuple you can access `FailedRows` to access the failed inserts. {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryIODeadLetter >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BigQueryIODeadLetter >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" BigQueryIODeadLetter >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" BigQueryIODeadLetter >}} {{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/custom-windows.md b/website/www/site/content/en/documentation/patterns/custom-windows.md index 22b5b285435d..23e83c46ada8 100644 --- a/website/www/site/content/en/documentation/patterns/custom-windows.md +++ b/website/www/site/content/en/documentation/patterns/custom-windows.md @@ -27,7 +27,7 @@ You can modify the [`assignWindows`](https://beam.apache.org/releases/javadoc/cu Access the `assignWindows` function through `WindowFn.AssignContext.element()`. The original, fixed-duration `assignWindows` function is: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow1 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow1 >}} {{< /highlight >}} ### Creating data-driven gaps @@ -38,13 +38,13 @@ To create data-driven gaps, add the following snippets to the `assignWindows` fu For example, the following function assigns each element to a window between the timestamp and `gapDuration`: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow3 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow3 >}} {{< /highlight >}} Then, set the `gapDuration` field in a windowing function: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow2 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow2 >}} {{< /highlight >}} ### Windowing messages into sessions @@ -53,13 +53,13 @@ After creating data-driven gaps, you can window incoming data into the new, cust First, set the session length to the gap duration: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow4 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow4 >}} {{< /highlight >}} Lastly, window data into sessions in your pipeline: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow6 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CustomSessionWindow6 >}} {{< /highlight >}} ### Example data and windows @@ -103,4 +103,4 @@ user=user-1, score=9, window=[2019-05-26T14:30:33.276Z..2019-05-26T14:30:41.849Z user=user-2, score=10, window=[2019-05-26T14:30:37.357Z..2019-05-26T14:30:47.357Z) ``` -With dynamic sessions, User #2 gets different scores. The third messages arrives seven seconds after the second message, so it's grouped into a different session. The large, 18-point session is split into two 9-point sessions. \ No newline at end of file +With dynamic sessions, User #2 gets different scores. The third messages arrives seven seconds after the second message, so it's grouped into a different session. The large, 18-point session is split into two 9-point sessions. diff --git a/website/www/site/content/en/documentation/patterns/file-processing.md b/website/www/site/content/en/documentation/patterns/file-processing.md index f119cd34da0a..d5d53dca558c 100644 --- a/website/www/site/content/en/documentation/patterns/file-processing.md +++ b/website/www/site/content/en/documentation/patterns/file-processing.md @@ -36,7 +36,7 @@ Use the [`FileIO`](https://beam.apache.org/releases/javadoc/current/org/apache/b {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternProcessNewFilesSnip1 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternProcessNewFilesSnip1 >}} {{< /highlight >}} {{< paragraph class="language-java" >}} @@ -44,7 +44,7 @@ The [`TextIO`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/ {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternProcessNewFilesSnip2 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternProcessNewFilesSnip2 >}} {{< /highlight >}} {{< paragraph class="language-java" >}} @@ -98,9 +98,9 @@ To read filenames in a pipeline job: {{< /paragraph >}} {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternAccessMetadataSnip1 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" FileProcessPatternAccessMetadataSnip1 >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" FileProcessPatternAccessMetadataSnip1 >}} -{{< /highlight >}} \ No newline at end of file +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" FileProcessPatternAccessMetadataSnip1 >}} +{{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/pipeline-options.md b/website/www/site/content/en/documentation/patterns/pipeline-options.md index 5bd379ea02f2..6cd1a7d14712 100644 --- a/website/www/site/content/en/documentation/patterns/pipeline-options.md +++ b/website/www/site/content/en/documentation/patterns/pipeline-options.md @@ -28,9 +28,9 @@ Use the `ValueProvider` interface to access runtime parameters after completing You can use the `ValueProvider` interface to pass runtime parameters to your pipeline, but you can only log the parameters from within the the Beam DAG. A solution is to add a pipeline [branch](/documentation/programming-guide/#applying-transforms) with a `DoFn` that processes a placeholder value and then logs the runtime parameters: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" AccessingValueProviderInfoAfterRunSnip1 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" AccessingValueProviderInfoAfterRunSnip1 >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" AccessingValueProviderInfoAfterRunSnip1 >}} -{{< /highlight >}} \ No newline at end of file +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" AccessingValueProviderInfoAfterRunSnip1 >}} +{{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/side-inputs.md b/website/www/site/content/en/documentation/patterns/side-inputs.md index 06cd2e6b247d..5d60acc20568 100644 --- a/website/www/site/content/en/documentation/patterns/side-inputs.md +++ b/website/www/site/content/en/documentation/patterns/side-inputs.md @@ -42,7 +42,7 @@ The global window side input triggers on processing time, so the main pipeline n For instance, the following code sample uses a `Map` to create a `DoFn`. The `Map` becomes a `View.asSingleton` side input that’s rebuilt on each counter tick. The side input updates every 5 seconds in order to demonstrate the workflow. In a real-world scenario, the side input would typically update every few hours or once per day. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SideInputPatternSlowUpdateGlobalWindowSnip1 >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SideInputPatternSlowUpdateGlobalWindowSnip1 >}} {{< /highlight >}} {{< highlight py >}} @@ -70,9 +70,9 @@ PCollection element. 1. Apply the side input. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" PeriodicallyUpdatingSideInputs >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" PeriodicallyUpdatingSideInputs >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" SideInputSlowUpdateSnip1 >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SideInputSlowUpdateSnip1 >}} {{< /highlight >}} diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 5f18e2ef0202..82bd0534633e 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -122,7 +122,7 @@ Pipeline p = Pipeline.create(options); {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_creating >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_creating >}} {{< /highlight >}} {{< highlight go >}} @@ -157,7 +157,7 @@ PipelineOptions options = {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_creating >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_creating >}} {{< /highlight >}} {{< highlight go >}} @@ -199,7 +199,7 @@ public interface MyOptions extends PipelineOptions { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_options_define_custom >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_options_define_custom >}} {{< /highlight >}} {{< highlight go >}} @@ -229,7 +229,7 @@ public interface MyOptions extends PipelineOptions { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_options_define_custom_with_help_and_default >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_options_define_custom_with_help_and_default >}} {{< /highlight >}} {{< highlight go >}} @@ -316,7 +316,7 @@ public static void main(String[] args) { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_reading >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipelines_constructing_reading >}} {{< /highlight >}} {{< highlight go >}} @@ -369,7 +369,7 @@ public static void main(String[] args) { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_pcollection >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_pcollection >}} {{< /highlight >}} ### 3.2. PCollection characteristics {#pcollection-characteristics} @@ -631,8 +631,8 @@ PCollection wordLengths = words.apply( words = ... # The DoFn to perform on each element in the input PCollection. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_pardo >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_apply >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_pardo >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_apply >}} {{< /highlight >}} {{< highlight go >}} @@ -708,7 +708,7 @@ static class ComputeWordLengthFn extends DoFn { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_pardo >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_pardo >}} {{< /highlight >}} {{< paragraph class="language-java" >}} @@ -771,7 +771,7 @@ words = ... # Apply a lambda function to the PCollection words. # Save the result as the PCollection word_lengths. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_using_flatmap >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_using_flatmap >}} {{< /highlight >}} {{< highlight go >}} @@ -810,7 +810,7 @@ words = ... # Apply a Map with a lambda function to the PCollection words. # Save the result as the PCollection word_lengths. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_using_map >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_using_map >}} {{< /highlight >}} {{< paragraph class="language-java" >}} @@ -966,22 +966,22 @@ data contains names and phone numbers. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleInputs >}} +{{< code_sample "examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleInputs >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_inputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_inputs >}} {{< /highlight >}} After `CoGroupByKey`, the resulting data contains all data associated with each unique key from any of the input collections. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleOutputs >}} +{{< code_sample "examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleOutputs >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_outputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_outputs >}} {{< /highlight >}} The following code example joins the two `PCollection`s with `CoGroupByKey`, @@ -989,21 +989,21 @@ followed by a `ParDo` to consume the result. Then, the code uses tags to look up and format data from each collection. {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CoGroupByKeyTuple >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" CoGroupByKeyTuple >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_group_by_key_cogroupbykey_tuple >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_group_by_key_cogroupbykey_tuple >}} {{< /highlight >}} The formatted data looks like this: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleFormattedOutputs >}} +{{< code_sample "examples/java/src/test/java/org/apache/beam/examples/snippets/SnippetsTest.java" CoGroupByKeyTupleFormattedOutputs >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_formatted_outputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_group_by_key_cogroupbykey_tuple_formatted_outputs >}} {{< /highlight >}} #### 4.2.4. Combine {#combine} @@ -1048,7 +1048,7 @@ public static class SumInts implements SerializableFunction, I {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_bounded_sum >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_bounded_sum >}} {{< /highlight >}} ##### 4.2.4.2. Advanced combinations using CombineFn {#advanced-combines} @@ -1121,7 +1121,7 @@ public class AverageFn extends CombineFn { {{< highlight py >}} pc = ... -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_custom_average_define >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_custom_average_define >}} {{< /highlight >}} ##### 4.2.4.3. Combining a PCollection into a single value {#combining-pcollection} @@ -1145,7 +1145,7 @@ PCollection sum = pc.apply( # The resulting PCollection, called result, contains one value: the sum of all # the elements in the input PCollection. pc = ... -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_custom_average_execute >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_custom_average_execute >}} {{< /highlight >}} ##### 4.2.4.4. Combine and global windowing {#combine-global-windowing} @@ -1229,7 +1229,7 @@ PCollection> avgAccuracyPerPlayer = # PCollection is grouped by key and the numeric values associated with each key # are averaged into a float. player_accuracies = ... -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_per_key >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" combine_per_key >}} {{< /highlight >}} #### 4.2.5. Flatten {#flatten} @@ -1258,7 +1258,7 @@ PCollection merged = collections.apply(Flatten.pCollections()); {{< highlight py >}} # Flatten takes a tuple of PCollection objects. # Returns a single PCollection that contains all of the elements in the PCollection objects in that tuple. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_flatten >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_flatten >}} {{< /highlight >}} ##### 4.2.5.1. Data encoding in merged collections {#data-encoding-merged-collections} @@ -1321,10 +1321,10 @@ PCollection fortiethPercentile = studentsByPercentile.get(4); # Provide an int value with the desired number of result partitions, and a partitioning function (partition_fn in this example). # Returns a tuple of PCollection objects containing each of the resulting partitions as individual PCollection objects. students = ... -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_partition >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_partition >}} # You can extract each partition from the tuple of PCollection objects as follows: -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_partition_40th >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_multiple_pcollections_partition_40th >}} {{< /highlight >}} ### 4.3. Requirements for writing user code for Beam transforms {#requirements-for-writing-user-code-for-beam-transforms} @@ -1448,12 +1448,12 @@ determined by the input data, or depend on a different branch of your pipeline. # of the actual elements of pcoll being passed into each process invocation. In this example, side inputs are # passed to a FlatMap transform as extra arguments and consumed by filter_using_length. words = ... -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_side_input >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_side_input >}} # We can also pass side inputs to a ParDo transform, which will get passed to its process method. # The first two arguments for the process method would be self and element. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_side_input_dofn >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_side_input_dofn >}} ... {{< /highlight >}} @@ -1550,12 +1550,12 @@ together. # with_outputs are attributes on the returned DoOutputsTuple object. The tags give access to the # corresponding output PCollections. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_tagged_outputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_tagged_outputs >}} # The result is also iterable, ordered in the same order that the tags were passed to with_outputs(), # the main tag (if specified) first. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_tagged_outputs_iter >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_tagged_outputs_iter >}} {{< /highlight >}} #### 4.5.2. Emitting to multiple outputs in your DoFn {#multiple-outputs-dofn} @@ -1588,12 +1588,12 @@ together. # using the pvalue.OutputValue wrapper class. # Based on the previous example, this shows the DoFn emitting to the main output and two additional outputs. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_emitting_values_on_tagged_outputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_emitting_values_on_tagged_outputs >}} # Producing multiple outputs is also available in Map and FlatMap. # Here is an example that uses FlatMap and shows that the tags do not need to be specified ahead of time. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_undeclared_outputs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_pardo_with_undeclared_outputs >}} {{< /highlight >}} #### 4.5.3. Accessing additional parameters in your DoFn {#other-dofn-parameters} @@ -1839,7 +1839,7 @@ transform's intermediate data changes type multiple times. {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_monitoring_composite >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" pipeline_monitoring_composite >}} {{< /highlight >}} #### 4.6.2. Creating a composite transform {#composite-transform-creation} @@ -1867,7 +1867,7 @@ The following code sample shows how to declare a `PTransform` that accepts a {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_transform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_transform >}} {{< /highlight >}} Within your `PTransform` subclass, you'll need to override the `expand` method. @@ -1891,7 +1891,7 @@ The following code sample shows how to override `expand` for the {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_transform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_transform >}} {{< /highlight >}} As long as you override the `expand` method in your `PTransform` subclass to @@ -1969,7 +1969,7 @@ p.apply("ReadFromText", {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_pipelineio_read >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_pipelineio_read >}} {{< /highlight >}} To read data from disparate sources into a single `PCollection`, read each one @@ -1994,7 +1994,7 @@ records.apply("WriteToText", {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" model_pipelineio_write >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_pipelineio_write >}} {{< /highlight >}} ### 5.4. Beam-provided I/O transforms {#provided-io-transforms} @@ -3660,7 +3660,7 @@ into fixed windows, each 60 seconds in length: {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_fixed_windows >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_fixed_windows >}} {{< /highlight >}} #### 8.3.2. Sliding time windows {#using-sliding-time-windows} @@ -3676,7 +3676,7 @@ begins every five seconds: {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_sliding_windows >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_sliding_windows >}} {{< /highlight >}} #### 8.3.3. Session windows {#using-session-windows} @@ -3692,7 +3692,7 @@ least 10 minutes (600 seconds): {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_session_windows >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_session_windows >}} {{< /highlight >}} Note that the sessions are per-key — each key in the collection will have its @@ -3711,7 +3711,7 @@ a single global window for a `PCollection`: {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_global_window >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_global_window >}} {{< /highlight >}} ### 8.4. Watermarks and late data {#watermarks-and-late-data} @@ -3822,7 +3822,7 @@ with a `DoFn` to attach the timestamps to each element in your `PCollection`. {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_timestamp >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" setting_timestamp >}} {{< /highlight >}} ## 9. Triggers {#triggers} @@ -3911,7 +3911,7 @@ firings: {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_early_late_triggers >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_early_late_triggers >}} {{< /highlight >}} #### 9.1.1. Default trigger {#default-trigger} @@ -3988,7 +3988,7 @@ sets the window's **accumulation mode**. {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_setting_trigger >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_setting_trigger >}} {{< /highlight >}} #### 9.4.1. Window accumulation modes {#window-accumulation-modes} @@ -4155,7 +4155,7 @@ example trigger code fires on the following conditions: {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_triggers >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_composite_triggers >}} {{< /highlight >}} #### 9.5.3. Other composite triggers {#other-composite-triggers} @@ -4171,7 +4171,7 @@ elements, or after a minute. {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" model_other_composite_triggers >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" model_other_composite_triggers >}} {{< /highlight >}} ## 10. Metrics {#metrics} @@ -4876,4 +4876,4 @@ perUser.apply(ParDo.of(new DoFn, OutputT>() { isTimerSetState.clear(); } })); -{{< /highlight >}} \ No newline at end of file +{{< /highlight >}} diff --git a/website/www/site/content/en/documentation/sdks/python-type-safety.md b/website/www/site/content/en/documentation/sdks/python-type-safety.md index 5147664ba94a..4b966db084c4 100644 --- a/website/www/site/content/en/documentation/sdks/python-type-safety.md +++ b/website/www/site/content/en/documentation/sdks/python-type-safety.md @@ -45,13 +45,13 @@ Introducing type hints for the `PTransforms` you define allows you to catch pote Consider the following example, in which `numbers` is a `PCollection` of `str` values: {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_missing_define_numbers >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_missing_define_numbers >}} {{< /highlight >}} The code then applies a `Filter` transform to the `numbers` collection with a callable that retrieves the even numbers. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_missing_apply >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_missing_apply >}} {{< /highlight >}} When you call `p.run()`, this code generates an error when trying to execute this transform because `Filter` expects a `PCollection` of integers, but is given a `PCollection` of strings instead. @@ -87,7 +87,7 @@ Annotations are currently supported on: The following code declares an `int` input and a `str` output type hint on the `to_id` transform, using annotations on `my_fn`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_map_annotations >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_map_annotations >}} {{< /highlight >}} The following code declares `int` input and output type hints on `filter_evens`, using annotations on `FilterEvensDoFn.process`. @@ -97,7 +97,7 @@ It is an error to have a non-iterable return type annotation for these functions Other supported iterable types include: `Iterator`, `Generator`, `Tuple`, `List`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_do_fn_annotations >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_do_fn_annotations >}} {{< /highlight >}} The following code declares `int` input and output type hints on `double_evens`, using annotations on `FilterEvensDoubleDoFn.process`. @@ -105,7 +105,7 @@ Since `process` returns a `list` or `None`, the output type is annotated as `Opt Beam will also remove the outer `Optional` and (as above) the outer iterable of the return type, only on the `DoFn.process` method and functions passed to `FlatMap`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_do_fn_annotations_optional >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test_py3.py" type_hints_do_fn_annotations_optional >}} {{< /highlight >}} ### Declaring Type Hints Inline @@ -113,7 +113,7 @@ Beam will also remove the outer `Optional` and (as above) the outer iterable of To specify type hints inline, use the methods `with_input_types` and `with_output_types`. The following example code declares an input type hint inline: {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_takes >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_takes >}} {{< /highlight >}} When you apply the Filter transform to the numbers collection in the example above, you'll be able to catch the error during pipeline construction. @@ -125,7 +125,7 @@ To specify type hints as properties of a `DoFn` or `PTransform`, use the decorat The following code declares an `int` type hint on `FilterEvensDoFn`, using the decorator `@with_input_types()`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_do_fn >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_do_fn >}} {{< /highlight >}} Decorators receive an arbitrary number of positional and/or keyword arguments, typically interpreted in the context of the function they're wrapping. Generally the first argument is a type hint for the main input, and additional arguments are type hints for side inputs. @@ -147,7 +147,7 @@ The following code specifies an input type hint that asserts the generic type `T If the input to `MyTransform` is of type `str`, Beam will infer the output type to be `Tuple[int, str]`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_transform >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_transform >}} {{< /highlight >}} ## Kinds of Type Hints @@ -200,13 +200,13 @@ In addition to using type hints for type checking at pipeline construction, you For example, the following pipeline emits elements of the wrong type. Depending on the runner implementation, its execution may or may not fail at runtime. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_runtime_off >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_runtime_off >}} {{< /highlight >}} However, if you enable runtime type checking, the code is guaranteed to fail at runtime. To enable runtime type checking, set the pipeline option `runtime_type_check` to `True`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_runtime_on >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_runtime_on >}} {{< /highlight >}} Note that because runtime type checks are done for each `PCollection` element, enabling this feature may incur a significant performance penalty. It is therefore recommended that runtime type checks are disabled for production pipelines. @@ -230,5 +230,5 @@ For example, suppose you have a `PCollection` of key-value pairs whose keys are The following code shows the example `Player` class and how to define a `Coder` for it. When you use type hints, Beam infers which `Coders` to use, using `beam.coders.registry`. The following code registers `PlayerCoder` as a coder for the `Player` class. In the example, the input type declared for `CombinePerKey` is `Tuple[Player, int]`. In this case, Beam infers that the `Coder` objects to use are `TupleCoder`, `PlayerCoder`, and `IntCoder`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_deterministic_key >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets_test.py" type_hints_deterministic_key >}} {{< /highlight >}} diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/filter.md b/website/www/site/content/en/documentation/transforms/python/elementwise/filter.md index 5d29149ba285..6a157f169452 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/filter.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/filter.md @@ -37,7 +37,7 @@ Then, we apply `Filter` in multiple ways to filter out produce by their duration We define a function `is_perennial` which returns `True` if the element's duration equals `'perennial'`, and `False` otherwise. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_function >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_function >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -45,7 +45,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} {{< /highlight >}} {{< buttons-code-snippet @@ -57,7 +57,7 @@ Output `PCollection` after `Filter`: We can also use lambda functions to simplify **Example 1**. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_lambda >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_lambda >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -65,7 +65,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} {{< /highlight >}} {{< buttons-code-snippet @@ -80,7 +80,7 @@ They are passed as additional positional arguments or keyword arguments to the f In this example, `has_duration` takes `plant` and `duration` as arguments. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_multiple_arguments >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_multiple_arguments >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -88,7 +88,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} {{< /highlight >}} {{< buttons-code-snippet @@ -104,7 +104,7 @@ In this example, we pass a `PCollection` the value `'perennial'` as a singleton. We then use that value to filter out perennials. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_singleton >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_singleton >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -112,7 +112,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} {{< /highlight >}} {{< buttons-code-snippet @@ -126,7 +126,7 @@ This accesses elements lazily as they are needed, so it is possible to iterate over large `PCollection`s that won't fit into memory. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_iter >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_iter >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -134,7 +134,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" valid_plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" valid_plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -152,7 +152,7 @@ Note that all the elements of the `PCollection` must fit into memory for this. If the `PCollection` won't fit into memory, use `beam.pvalue.AsIter(pcollection)` instead. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_dict >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter.py" filter_side_inputs_dict >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -160,7 +160,7 @@ Output `PCollection` after `Filter`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/filter_test.py" perennials >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/flatmap.md b/website/www/site/content/en/documentation/transforms/python/elementwise/flatmap.md index 5fbdad6d88cf..872805b38637 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/flatmap.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/flatmap.md @@ -38,7 +38,7 @@ We use the function `str.split` which takes a single `str` element and outputs a This pipeline splits the input element using whitespaces, creating a list of zero or more elements. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_simple >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_simple >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -46,7 +46,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -58,7 +58,7 @@ Output `PCollection` after `FlatMap`: We define a function `split_words` which splits an input `str` element using the delimiter `','` and outputs a `list` of `str`s. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_function >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_function >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -66,7 +66,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} @@ -81,7 +81,7 @@ Each input element is already an `iterable`, where each element is what we want We use a lambda function that returns the same input element it received. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_lambda >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_lambda >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -89,7 +89,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -103,7 +103,7 @@ We use a generator to iterate over the input list and yield each of the elements Each yielded result in the generator is an element in the resulting `PCollection`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_generator >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_generator >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -111,7 +111,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -124,7 +124,7 @@ If your `PCollection` consists of `(key, value)` pairs, you can use `FlatMapTuple` to unpack them into different function arguments. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_tuple >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_tuple >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -132,7 +132,7 @@ Output `PCollection` after `FlatMapTuple`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -147,7 +147,7 @@ They are passed as additional positional arguments or keyword arguments to the f In this example, `split_words` takes `text` and `delimiter` as arguments. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_multiple_arguments >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_multiple_arguments >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -155,7 +155,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -171,7 +171,7 @@ In this example, we pass a `PCollection` the value `','` as a singleton. We then use that value as the delimiter for the `str.split` method. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_singleton >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_singleton >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -179,7 +179,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -193,7 +193,7 @@ This accesses elements lazily as they are needed, so it is possible to iterate over large `PCollection`s that won't fit into memory. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_iter >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_iter >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -201,7 +201,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" valid_plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" valid_plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -219,7 +219,7 @@ Note that all the elements of the `PCollection` must fit into memory for this. If the `PCollection` won't fit into memory, use `beam.pvalue.AsIter(pcollection)` instead. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_dict >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap.py" flatmap_side_inputs_dict >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -227,7 +227,7 @@ Output `PCollection` after `FlatMap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" valid_plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/flatmap_test.py" valid_plants >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/keys.md b/website/www/site/content/en/documentation/transforms/python/elementwise/keys.md index 013e36e7a504..65b4cba33266 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/keys.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/keys.md @@ -29,7 +29,7 @@ In the following example, we create a pipeline with a `PCollection` of key-value Then, we apply `Keys` to extract the keys and discard the values. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys.py" keys >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys.py" keys >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -37,7 +37,7 @@ Output `PCollection` after `Keys`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py" icons >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/keys_test.py" icons >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/kvswap.md b/website/www/site/content/en/documentation/transforms/python/elementwise/kvswap.md index 298cbaaf37ea..0bfe13df1baa 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/kvswap.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/kvswap.md @@ -30,7 +30,7 @@ In the following example, we create a pipeline with a `PCollection` of key-value Then, we apply `KvSwap` to swap the keys and values. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap.py" kvswap >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap.py" kvswap >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -38,7 +38,7 @@ Output `PCollection` after `KvSwap`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/kvswap_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/map.md b/website/www/site/content/en/documentation/transforms/python/elementwise/map.md index 0a0c31ff6e69..f3036fea0c5b 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/map.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/map.md @@ -36,7 +36,7 @@ We use the function `str.strip` which takes a single `str` element and outputs a It strips the input element's whitespaces, including newlines and tabs. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_simple >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_simple >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -44,7 +44,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -56,7 +56,7 @@ Output `PCollection` after `Map`: We define a function `strip_header_and_newline` which strips any `'#'`, `' '`, and `'\n'` characters from each element. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_function >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_function >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -64,7 +64,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -76,7 +76,7 @@ Output `PCollection` after `Map`: We can also use lambda functions to simplify **Example 2**. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_lambda >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_lambda >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -84,7 +84,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -99,7 +99,7 @@ They are passed as additional positional arguments or keyword arguments to the f In this example, `strip` takes `text` and `chars` as arguments. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_multiple_arguments >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_multiple_arguments >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -107,7 +107,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -120,7 +120,7 @@ If your `PCollection` consists of `(key, value)` pairs, you can use `MapTuple` to unpack them into different function arguments. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_tuple >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_tuple >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -128,7 +128,7 @@ Output `PCollection` after `MapTuple`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -144,7 +144,7 @@ In this example, we pass a `PCollection` the value `'# \n'` as a singleton. We then use that value as the characters for the `str.strip` method. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_singleton >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_singleton >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -152,7 +152,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -166,7 +166,7 @@ This accesses elements lazily as they are needed, so it is possible to iterate over large `PCollection`s that won't fit into memory. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_iter >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_iter >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -174,7 +174,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -192,7 +192,7 @@ Note that all the elements of the `PCollection` must fit into memory for this. If the `PCollection` won't fit into memory, use `beam.pvalue.AsIter(pcollection)` instead. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_dict >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map.py" map_side_inputs_dict >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -200,7 +200,7 @@ Output `PCollection` after `Map`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plant_details >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/map_test.py" plant_details >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md b/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md index c0514693954a..cd10cdeae67a 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/pardo.md @@ -42,7 +42,7 @@ The `process` method is called once per element, and it can yield zero or more output elements. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn >}} {{}} {{< paragraph class="notebook-skip" >}} @@ -50,7 +50,7 @@ Output `PCollection` after `ParDo`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -71,7 +71,7 @@ In this example, we add new parameters to the `process` method to bind parameter object. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn_params >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn_params >}} {{}} {{< paragraph class="notebook-skip" >}} @@ -79,7 +79,7 @@ In this example, we add new parameters to the `process` method to bind parameter {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" dofn_params >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" dofn_params >}} {{< /highlight >}} {{< buttons-code-snippet @@ -132,7 +132,7 @@ starts and finishes with `start_bundle` and `finish_bundle`. For example, if the worker crashes, `teardown` might not be called. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn_methods >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo.py" pardo_dofn_methods >}} {{}} {{< paragraph class="notebook-skip" >}} @@ -140,7 +140,7 @@ starts and finishes with `start_bundle` and `finish_bundle`. {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" results >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/pardo_test.py" results >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/partition.md b/website/www/site/content/en/documentation/transforms/python/elementwise/partition.md index 5034086cd763..72f6aa0502bd 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/partition.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/partition.md @@ -47,7 +47,7 @@ In the following example, we have a known list of durations. We partition the `PCollection` into one `PCollection` for every duration type. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_function >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_function >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -55,7 +55,7 @@ Output `PCollection`s: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" partitions >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" partitions >}} {{< /highlight >}} {{< buttons-code-snippet @@ -67,7 +67,7 @@ Output `PCollection`s: We can also use lambda functions to simplify **Example 1**. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_lambda >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_lambda >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -75,7 +75,7 @@ Output `PCollection`s: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" partitions >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" partitions >}} {{< /highlight >}} {{< buttons-code-snippet @@ -115,7 +115,7 @@ This `split_dataset` function is generic enough to support any number of partiti You might want to adapt the bucket assignment to use a more appropriate or randomized hash for your dataset. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_multiple_arguments >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition.py" partition_multiple_arguments >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -123,7 +123,7 @@ Output `PCollection`s: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" train_test >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/partition_test.py" train_test >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/regex.md b/website/www/site/content/en/documentation/transforms/python/elementwise/regex.md index 3374f4c25d48..2e55c1589257 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/regex.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/regex.md @@ -62,7 +62,7 @@ To start matching at any point instead of the beginning of the string, use [`Regex.find(regex)`](#example-4-regex-find). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_matches >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_matches >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -70,7 +70,7 @@ Output `PCollection` after `Regex.matches`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches >}} {{< /highlight >}} {{< buttons-code-snippet @@ -91,7 +91,7 @@ To start matching at any point instead of the beginning of the string, use [`Regex.find_all(regex, group=Regex.ALL, outputEmpty=False)`](#example-5-regex-find-all). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_all_matches >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_all_matches >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -99,7 +99,7 @@ Output `PCollection` after `Regex.all_matches`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_all_matches >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_all_matches >}} {{< /highlight >}} {{< buttons-code-snippet @@ -121,7 +121,7 @@ To start matching at any point instead of the beginning of the string, use [`Regex.find_kv(regex, keyGroup)`](#example-6-regex-find-as-key-value-pairs). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_matches_kv >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_matches_kv >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -129,7 +129,7 @@ Output `PCollection` after `Regex.matches_kv`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches_kv >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches_kv >}} {{< /highlight >}} {{< buttons-code-snippet @@ -151,7 +151,7 @@ If you need to match from the start only, consider using [`Regex.matches(regex)`](#example-1-regex-match). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -159,7 +159,7 @@ Output `PCollection` after `Regex.find`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_matches >}} {{< /highlight >}} {{< buttons-code-snippet @@ -181,7 +181,7 @@ If you need to match all groups from the start only, consider using [`Regex.all_matches(regex)`](#example-2-regex-match-with-all-groups). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find_all >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find_all >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -189,7 +189,7 @@ Output `PCollection` after `Regex.find_all`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_find_all >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_find_all >}} {{< /highlight >}} {{< buttons-code-snippet @@ -212,7 +212,7 @@ If you need to match as key-value pairs from the start only, consider using [`Regex.matches_kv(regex)`](#example-3-regex-match-into-key-value-pairs). {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find_kv >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_find_kv >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -220,7 +220,7 @@ Output `PCollection` after `Regex.find_kv`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_find_kv >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_find_kv >}} {{< /highlight >}} {{< buttons-code-snippet @@ -235,7 +235,7 @@ You can also use on the `replacement`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_replace_all >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_replace_all >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -243,7 +243,7 @@ Output `PCollection` after `Regex.replace_all`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_replace_all >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_replace_all >}} {{< /highlight >}} {{< buttons-code-snippet @@ -258,7 +258,7 @@ You can also use on the `replacement`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_replace_first >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_replace_first >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -266,7 +266,7 @@ Output `PCollection` after `Regex.replace_first`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_replace_first >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_replace_first >}} {{< /highlight >}} {{< buttons-code-snippet @@ -279,7 +279,7 @@ Output `PCollection` after `Regex.replace_first`: The argument `outputEmpty` is set to `False` by default, but can be set to `True` to keep empty items in the output list. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_split >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex.py" regex_split >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -287,7 +287,7 @@ Output `PCollection` after `Regex.split`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_split >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/regex_test.py" plants_split >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/tostring.md b/website/www/site/content/en/documentation/transforms/python/elementwise/tostring.md index 292825c41ba7..61a2070fd388 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/tostring.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/tostring.md @@ -36,7 +36,7 @@ The following example converts a `(key, value)` pair into a string delimited by You can specify a different delimiter using the `delimiter` argument. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_kvs >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_kvs >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -44,7 +44,7 @@ Output `PCollection` after `ToString`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet @@ -57,7 +57,7 @@ The following example converts a dictionary into a string. The string output will be equivalent to `str(element)`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_element >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_element >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -65,7 +65,7 @@ Output `PCollection` after `ToString`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plant_lists >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plant_lists >}} {{< /highlight >}} {{< buttons-code-snippet @@ -80,7 +80,7 @@ You can specify a different delimiter using the `delimiter` argument. The string output will be equivalent to `iterable.join(delimiter)`. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_iterables >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring.py" tostring_iterables >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -88,7 +88,7 @@ Output `PCollection` after `ToString`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plants_csv >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/tostring_test.py" plants_csv >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/values.md b/website/www/site/content/en/documentation/transforms/python/elementwise/values.md index b1179596da93..11e54a4fdbc7 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/values.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/values.md @@ -29,7 +29,7 @@ In the following example, we create a pipeline with a `PCollection` of key-value Then, we apply `Values` to extract the values and discard the keys. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values.py" values >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/values.py" values >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -37,7 +37,7 @@ Output `PCollection` after `Values`: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py" plants >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/values_test.py" plants >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/withtimestamps.md b/website/www/site/content/en/documentation/transforms/python/elementwise/withtimestamps.md index 3a1c6fd9a10b..3cc037217b3f 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/withtimestamps.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/withtimestamps.md @@ -34,7 +34,7 @@ The elements themselves often already contain a timestamp field. in the form of seconds. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_event_time >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_event_time >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -42,7 +42,7 @@ Output `PCollection` after getting the timestamps: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_timestamps >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_timestamps >}} {{< /highlight >}} {{< buttons-code-snippet @@ -57,7 +57,7 @@ For more information on time formatting options, see [`time.strftime`](https://docs.python.org/3/library/time.html#time.strftime). {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" time_tuple2unix_time >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" time_tuple2unix_time >}} {{< /highlight >}} To convert from a @@ -66,7 +66,7 @@ to `unix_time` you can use convert it to a `time.struct_time` first with [`datetime.timetuple`](https://docs.python.org/3/library/datetime.html#datetime.datetime.timetuple). {{< highlight >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" datetime2unix_time >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" datetime2unix_time >}} {{< /highlight >}} ### Example 2: Timestamp by logical clock @@ -76,7 +76,7 @@ If each element has a chronological number, these numbers can be used as a These numbers have to be converted to a *"seconds"* equivalent, which can be especially important depending on your windowing and late data rules. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_logical_clock >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_logical_clock >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -84,7 +84,7 @@ Output `PCollection` after getting the timestamps: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_events >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_events >}} {{< /highlight >}} {{< buttons-code-snippet @@ -100,7 +100,7 @@ Workers might have time deltas, so using this method is not a reliable way to do By using processing time, there is no way of knowing if data is arriving late because the timestamp is attached when the element *enters* into the pipeline. {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_processing_time >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps.py" withtimestamps_processing_time >}} {{< /highlight >}} {{< paragraph class="notebook-skip" >}} @@ -108,7 +108,7 @@ Output `PCollection` after getting the timestamps: {{< /paragraph >}} {{< highlight class="notebook-skip" >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_processing_times >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/withtimestamps_test.py" plant_processing_times >}} {{< /highlight >}} {{< buttons-code-snippet diff --git a/website/www/site/content/en/get-started/mobile-gaming-example.md b/website/www/site/content/en/get-started/mobile-gaming-example.md index 9054f5373e10..6a976c56f42f 100644 --- a/website/www/site/content/en/get-started/mobile-gaming-example.md +++ b/website/www/site/content/en/get-started/mobile-gaming-example.md @@ -101,11 +101,11 @@ This example uses batch processing, and the diagram's Y axis represents processi After reading the score events from the input file, the pipeline groups all of those user/score pairs together and sums the score values into one total value per unique user. `UserScore` encapsulates the core logic for that step as the [user-defined composite transform](/documentation/programming-guide/#composite-transforms) `ExtractAndSumScore`: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java" DocInclude_USExtractXform >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java" DocInclude_USExtractXform >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/user_score.py" extract_and_sum_score >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/user_score.py" extract_and_sum_score >}} {{< /highlight >}} `ExtractAndSumScore` is written to be more general, in that you can pass in the field by which you want to group the data (in the case of our game, by unique user or unique team). This means we can re-use `ExtractAndSumScore` in other pipelines that group score data by team, for example. @@ -113,11 +113,11 @@ After reading the score events from the input file, the pipeline groups all of t Here's the main method of `UserScore`, showing how we apply all three steps of the pipeline: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java" DocInclude_USMain >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java" DocInclude_USMain >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/user_score.py" main >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/user_score.py" main >}} {{< /highlight >}} ### Limitations @@ -186,11 +186,11 @@ Beam's windowing feature uses the [intrinsic timestamp information](/documentati The following code shows this: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSAddTsAndWindow >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSAddTsAndWindow >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" add_timestamp_and_window >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" add_timestamp_and_window >}} {{< /highlight >}} Notice that the transforms the pipeline uses to specify the windowing are distinct from the actual data processing transforms (such as `ExtractAndSumScores`). This functionality provides you some flexibility in designing your Beam pipeline, in that you can run existing transforms over datasets with different windowing characteristics. @@ -206,11 +206,11 @@ It also lets the pipeline include relevant **late data**—data events with vali The following code shows how `HourlyTeamScore` uses the `Filter` transform to filter events that occur either before or after the relevant analysis period: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSFilters >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSFilters >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" filter_by_time_range >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" filter_by_time_range >}} {{< /highlight >}} #### Calculating Score Per Team, Per Window @@ -218,11 +218,11 @@ The following code shows how `HourlyTeamScore` uses the `Filter` transform to fi `HourlyTeamScore` uses the same `ExtractAndSumScores` transform as the `UserScore` pipeline, but passes a different key (team, as opposed to user). Also, because the pipeline applies `ExtractAndSumScores` _after_ applying fixed-time 1-hour windowing to the input data, the data gets grouped by both team _and_ window. You can see the full sequence of transforms in `HourlyTeamScore`'s main method: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSMain >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java" DocInclude_HTSMain >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" main >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/hourly_team_score.py" main >}} {{< /highlight >}} ### Limitations @@ -279,11 +279,11 @@ As processing time advances and more scores are processed, the trigger outputs t The following code example shows how `LeaderBoard` sets the processing time trigger to output the data for user scores: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java" DocInclude_ProcTimeTrigger >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java" DocInclude_ProcTimeTrigger >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/leader_board.py" processing_time_trigger >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/leader_board.py" processing_time_trigger >}} {{< /highlight >}} `LeaderBoard` sets the [window accumulation mode](/documentation/programming-guide/#window-accumulation-modes) to accumulate window panes as the trigger fires. This accumulation mode is set by invoking `.accumulatingFiredPanes` using `accumulation_mode=trigger.AccumulationMode.ACCUMULATING` when setting the trigger, and causes the pipeline to accumulate the previously emitted data together with any new data that's arrived since the last trigger fire. This ensures that `LeaderBoard` is a running sum for the user scores, rather than a collection of individual sums. @@ -313,11 +313,11 @@ Data arriving above the solid watermark line is _late data_ — this is a score The following code example shows how `LeaderBoard` applies fixed-time windowing with the appropriate triggers to have our pipeline perform the calculations we want: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java" DocInclude_WindowAndTrigger >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java" DocInclude_WindowAndTrigger >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/leader_board.py" window_and_trigger >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/leader_board.py" window_and_trigger >}} {{< /highlight >}} Taken together, these processing strategies let us address the latency and completeness issues present in the `UserScore` and `HourlyTeamScore` pipelines, while still using the same basic transforms to process the data—as a matter of fact, both calculations still use the same `ExtractAndSumScore` transform that we used in both the `UserScore` and `HourlyTeamScore` pipelines. @@ -356,21 +356,21 @@ Since the average depends on the pipeline data, we need to calculate it, and the The following code example shows the composite transform that handles abuse detection. The transform uses the `Sum.integersPerKey` transform to sum all scores per user, and then the `Mean.globally` transform to determine the average score for all users. Once that's been calculated (as a `PCollectionView` singleton), we can pass it to the filtering `ParDo` using `.withSideInputs`: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_AbuseDetect >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_AbuseDetect >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/game_stats.py" abuse_detect >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/game_stats.py" abuse_detect >}} {{< /highlight >}} The abuse-detection transform generates a view of users supected to be spambots. Later in the pipeline, we use that view to filter out any such users when we calculate the team score per hour, again by using the side input mechanism. The following code example shows where we insert the spam filter, between windowing the scores into fixed windows and extracting the team scores: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_FilterAndCalc >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_FilterAndCalc >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/game_stats.py" filter_and_calc >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/game_stats.py" filter_and_calc >}} {{< /highlight >}} #### Analyzing Usage Patterns @@ -390,21 +390,21 @@ between instances are.* We can use the session-windowed data to determine the average length of uninterrupted play time for all of our users, as well as the total score they achieve during each session. We can do this in the code by first applying session windows, summing the score per user and session, and then using a transform to calculate the length of each individual session: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_SessionCalc >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_SessionCalc >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/game_stats.py" session_calc >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/game_stats.py" session_calc >}} {{< /highlight >}} This gives us a set of user sessions, each with an attached duration. We can then calculate the _average_ session length by re-windowing the data into fixed time windows, and then calculating the average for all sessions that end in each hour: {{< highlight java >}} -{{< github_sample "/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_Rewindow >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/complete/game/GameStats.java" DocInclude_Rewindow >}} {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/complete/game/game_stats.py" rewindow >}} +{{< code_sample "sdks/python/apache_beam/examples/complete/game/game_stats.py" rewindow >}} {{< /highlight >}} We can use the resulting information to find, for example, what times of day our users are playing the longest, or which stretches of the day are more likely to see shorter play sessions. diff --git a/website/www/site/content/en/get-started/wordcount-example.md b/website/www/site/content/en/get-started/wordcount-example.md index 58ae28507e34..08e447912fc6 100644 --- a/website/www/site/content/en/get-started/wordcount-example.md +++ b/website/www/site/content/en/get-started/wordcount-example.md @@ -123,7 +123,7 @@ sections, we will specify the pipeline's runner. {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_options >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_options >}} {{< /highlight >}} {{< paragraph class="language-java language-py" >}} @@ -143,7 +143,7 @@ Pipeline p = Pipeline.create(options); {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_create >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_create >}} {{< /highlight >}} {{< highlight go >}} @@ -179,7 +179,7 @@ p.apply(TextIO.read().from("gs://apache-beam-samples/shakespeare/*")) {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_read >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_read >}} {{< /highlight >}} {{< highlight go >}} @@ -204,7 +204,7 @@ lines := textio.Read(s, "gs://apache-beam-samples/shakespeare/*") {{< highlight py >}} # The Flatmap transform is a simplified version of ParDo. -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_pardo >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_pardo >}} {{< /highlight >}} {{< highlight go >}} @@ -231,7 +231,7 @@ words := beam.ParDo(s, func(line string, emit func(string)) { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_count >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_count >}} {{< /highlight >}} {{< highlight go >}} @@ -253,7 +253,7 @@ counted := stats.Count(s, words) {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_map >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_map >}} {{< /highlight >}} {{< highlight go >}} @@ -272,7 +272,7 @@ formatted := beam.ParDo(s, func(w string, c int) string { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_write >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_write >}} {{< /highlight >}} {{< highlight go >}} @@ -304,7 +304,7 @@ p.run().waitUntilFinish(); {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_run >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_minimal_run >}} {{< /highlight >}} {{< highlight go >}} @@ -534,7 +534,7 @@ static class ExtractWordsFn extends DoFn { {{< highlight py >}} # In this example, the DoFns are defined as classes: -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_dofn >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_dofn >}} {{< /highlight >}} {{< highlight go >}} @@ -603,7 +603,7 @@ public static void main(String[] args) throws IOException { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_composite >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_composite >}} {{< /highlight >}} {{< highlight go >}} @@ -652,7 +652,7 @@ public static void main(String[] args) { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_options >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" examples_wordcount_wordcount_options >}} {{< /highlight >}} {{< highlight go >}} @@ -862,7 +862,7 @@ public class DebuggingWordCount { {{< /highlight >}} {{< highlight py >}} -{{< github_sample "/apache/beam/blob/master/sdks/python/apache_beam/examples/snippets/snippets.py" example_wordcount_debugging_logging >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" example_wordcount_debugging_logging >}} {{< /highlight >}} {{< highlight go >}} diff --git a/website/www/site/layouts/shortcodes/code_sample.html b/website/www/site/layouts/shortcodes/code_sample.html new file mode 100644 index 000000000000..4a94cc2e5346 --- /dev/null +++ b/website/www/site/layouts/shortcodes/code_sample.html @@ -0,0 +1,23 @@ +{{/* + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. See accompanying LICENSE file. +*/}} +{{/* + This shortcode is used to fetch a piece of code with tags from Beam code. + In setupDockerContainer stage of build process, we run build_code_samples.sh script, which + copies Beam project files from which we take code snippets to website/www/site/code_samples + directory. We do that because Hugo can't access files which are outside of its project tree + (Hugo's root is the directory that contains config.toml). When copying, we name this files + according to convention path_to_file_filename.extension in order to avoid name conflicts. + .Get 0 references an argument passed to code_sample shortcode which is a path to file with + code snippet. So if we pass path/to/file/filename.py to code_sample, $path variable will + have value code_samples/path_to_file_filename.py which Hugo can access. + There should be no breaklines here to make sure the string results do not get impacts of newlines. +*/}}{{ $tag := .Get 1 }}{{ $path := printf "code_samples/%s" (replaceRE "/" "_" (.Get 0)) }}{{ $data := readFile $path }}{{ $matchRegex := printf "%s%s%s%s%s" "\\[START " $tag "]\n[\\s\\S]*?\n.*\\[END " $tag "]" }}{{ $match := index (findRE $matchRegex $data) 0 }}{{ $lines := split $match "\n" }}{{ $lineCount := len $lines }}{{ $cleanedLines := $lines | first (sub $lineCount 1) | last (sub $lineCount 2) }}{{ $firstLine := index $cleanedLines 0 }}{{ $numberOfWhitespaces := index (findRE "^\\s*" $firstLine) 0 | len }}{{ $unindentRegex := printf "%s%d%s" "^\\s{" $numberOfWhitespaces "}" }}{{ $unindentedLines := apply $cleanedLines "replaceRE" $unindentRegex "" "." }}{{ $result := delimit $unindentedLines "\n" }}{{ print $result }} diff --git a/website/www/site/layouts/shortcodes/github_sample.html b/website/www/site/layouts/shortcodes/github_sample.html deleted file mode 100644 index db9fe9319687..000000000000 --- a/website/www/site/layouts/shortcodes/github_sample.html +++ /dev/null @@ -1,15 +0,0 @@ -{{/* - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. See accompanying LICENSE file. -*/}} -{{/* - This shortcode is used to fetch a piece of code with tags from GitHub. - There should be no breaklines here to make sure the string results do not get impacts of newlines. -*/}}{{ $tag := .Get 1 }}{{ $path := replaceRE "/blob/" "/" (.Get 0) }}{{ $path := printf "github_samples/%s" (replaceRE "/" "_" $path) }}{{ $data := readFile $path }}{{ $matchRegex := printf "%s%s%s%s%s" "\\[START " $tag "]\n[\\s\\S]*?\n.*\\[END " $tag "]" }}{{ $match := index (findRE $matchRegex $data) 0 }}{{ $lines := split $match "\n" }}{{ $lineCount := len $lines }}{{ $cleanedLines := $lines | first (sub $lineCount 1) | last (sub $lineCount 2) }}{{ $firstLine := index $cleanedLines 0 }}{{ $numberOfWhitespaces := index (findRE "^\\s*" $firstLine) 0 | len }}{{ $unindentRegex := printf "%s%d%s" "^\\s{" $numberOfWhitespaces "}" }}{{ $unindentedLines := apply $cleanedLines "replaceRE" $unindentRegex "" "." }}{{ $result := delimit $unindentedLines "\n" }}{{ print $result }} From 4817e843ad8e5aa8a8831a0fc0e2c5f10b0ab76f Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Wed, 27 May 2020 18:49:37 -0400 Subject: [PATCH 103/151] [BEAM-9971] Do not use context classloader. (#11784) * [BEAM-9971] Do not use context classloader. * Add TODO(BEAM-10109). --- .../beam/runners/core/construction/NativeTransforms.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java index 730a5ca561f5..9ea47a10835b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java @@ -46,7 +46,10 @@ public class NativeTransforms { * Returns true if an only if the Runner understands this transform and can handle it directly. */ public static boolean isNative(RunnerApi.PTransform pTransform) { - Iterator matchers = ServiceLoader.load(IsNativeTransform.class).iterator(); + // TODO(BEAM-10109) Use default (context) classloader. + Iterator matchers = + ServiceLoader.load(IsNativeTransform.class, NativeTransforms.class.getClassLoader()) + .iterator(); while (matchers.hasNext()) { if (matchers.next().test(pTransform)) { return true; From d17259491a666b0bf7e11b4f5001f8d747aa06cc Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Wed, 27 May 2020 18:21:45 -0700 Subject: [PATCH 104/151] Enables proto holders for testing runner API. Without this x-lang can be broken for some execution paths where this property get enabled automatically. --- sdks/python/apache_beam/pipeline.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index d1380c6294b7..9d1b02c53e01 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -509,7 +509,8 @@ def run(self, test_runner_api='AUTO'): return Pipeline.from_runner_api( self.to_runner_api(use_fake_coders=True), self.runner, - self._options).run(False) + self._options, + allow_proto_holders=True).run(False) if self._options.view_as(TypeOptions).runtime_type_check: from apache_beam.typehints import typecheck From 05882606bc638698ebc8e16a978d184b3f7ba5fe Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Wed, 27 May 2020 20:54:57 -0700 Subject: [PATCH 105/151] [BEAM-9363] Support TUMBLE aggregation (#11807) * [BEAM-9363] Support TUMBLE aggregation. --- .../impl/rel/BeamTableFunctionScanRel.java | 37 +++++++++++++++---- .../sql/zetasql/ZetaSQLDialectSpecTest.java | 25 +++++++++++++ 2 files changed, 54 insertions(+), 8 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java index 74d6ccae41e4..6381fd96b787 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java @@ -29,8 +29,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.WithTimestamps; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.Row; @@ -99,14 +102,32 @@ public PCollection expand(PCollectionList input) { RexInputRef wmCol = (RexInputRef) call.getOperands().get(1); PCollection upstream = input.get(0); Schema outputSchema = CalciteUtils.toSchema(getRowType()); - return upstream - .apply( - ParDo.of( - new FixedWindowDoFn( - FixedWindows.of(durationParameter(call.getOperands().get(2))), - wmCol.getIndex(), - outputSchema))) - .setRowSchema(outputSchema); + FixedWindows windowFn = FixedWindows.of(durationParameter(call.getOperands().get(2))); + PCollection streamWithWindowMetadata = + upstream + .apply(ParDo.of(new FixedWindowDoFn(windowFn, wmCol.getIndex(), outputSchema))) + .setRowSchema(outputSchema); + + PCollection windowedStream = + assignTimestampsAndWindow( + streamWithWindowMetadata, wmCol.getIndex(), (WindowFn) windowFn); + + return windowedStream; + } + + /** Extract timestamps from the windowFieldIndex, then window into windowFns. */ + private PCollection assignTimestampsAndWindow( + PCollection upstream, int windowFieldIndex, WindowFn windowFn) { + PCollection windowedStream; + windowedStream = + upstream + .apply( + "assignEventTimestamp", + WithTimestamps.of(row -> row.getDateTime(windowFieldIndex).toInstant()) + .withAllowedTimestampSkew(new Duration(Long.MAX_VALUE))) + .setCoder(upstream.getCoder()) + .apply(Window.into(windowFn)); + return windowedStream; } } diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java index 8385c706315a..c6aa39b1ef37 100644 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java +++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java @@ -4780,6 +4780,31 @@ public void testTumbleAsTVF() { pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } + @Test + public void testTVFTumbleAggregation() { + String sql = + "SELECT COUNT(*) as field_count, " + + "window_start " + + "FROM TUMBLE((select * from KeyValue), descriptor(ts), 'INTERVAL 1 SECOND') " + + "GROUP BY window_start"; + ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); + BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); + + PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); + + final Schema schema = + Schema.builder().addInt64Field("field_count").addDateTimeField("window_start").build(); + PAssert.that(stream) + .containsInAnyOrder( + Row.withSchema(schema) + .addValues(1L, new DateTime(2018, 7, 1, 21, 26, 7, ISOChronology.getInstanceUTC())) + .build(), + Row.withSchema(schema) + .addValues(1L, new DateTime(2018, 7, 1, 21, 26, 6, ISOChronology.getInstanceUTC())) + .build()); + pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); + } + @Test public void testIsNullTrueFalse() { String sql = From d71ee351e84f83c29b8d0759c9dcde0b477f161a Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Thu, 28 May 2020 13:31:15 +0200 Subject: [PATCH 106/151] [BEAM-9916] Fixed some IO Urls on website --- website/www/site/data/io_matrix.yaml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml index f1e4c962f0e0..3ce5a4f864ec 100644 --- a/website/www/site/data/io_matrix.yaml +++ b/website/www/site/data/io_matrix.yaml @@ -61,13 +61,13 @@ categories: implementations: - language: java name: org.apache.beam.sdk.io.xml.XmlIO - url: https://beam.apache.org/releases/javadoc/2.3.0/org/apache/beam/sdk/io/xml/XmlIO.html + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/xml/XmlIO.html - transform: TikaIO description: Transforms for parsing arbitrary files using [Apache Tika](https://tika.apache.org/). implementations: - language: java name: org.apache.beam.sdk.io.tika.TikaIO - url: https://beam.apache.org/releases/javadoc/2.3.0/org/apache/beam/sdk/io/tika/TikaIO.html + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/tika/TikaIO.html - transform: ParquetIO description: IO for reading from and writing to [Parquet](https://parquet.apache.org/) files. docs: /documentation/io/built-in/parquet/ @@ -365,13 +365,13 @@ categories: implementations: - language: py name: apache_beam.io.flink.flink_streaming_impulse_source - url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.flink.flink_streaming_impulse_source.html + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.flink.flink_streaming_impulse_source.html - transform: GenerateSequence description: Generates a bounded or unbounded stream of integers. implementations: - language: java name: org.apache.beam.sdk.io.GenerateSequence - url: https://beam.apache.org/releases/javadoc/2.20.0/org/apache/beam/sdk/io/GenerateSequence.html + url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/GenerateSequence.html - language: py name: apache_beam.io.external.generate_sequence.GenerateSequence - url: https://beam.apache.org/releases/pydoc/2.20.0/apache_beam.io.external.generate_sequence.html + url: https://beam.apache.org/releases/pydoc/current/apache_beam.io.external.generate_sequence.html From 972c331f12b74efbe127cf8e1befe9b5de8b292f Mon Sep 17 00:00:00 2001 From: Steven Niemitz Date: Thu, 28 May 2020 12:17:23 -0400 Subject: [PATCH 107/151] Avoid re-encoding value state cells each commit (#11823) --- .../worker/WindmillStateInternals.java | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java index a3619ab9bfab..964245b43715 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java @@ -345,6 +345,8 @@ private static class WindmillValue extends SimpleWindmillState implements Val private boolean modified = false; /** Whether the in memory value is the true value. */ private boolean valueIsKnown = false; + /** The size of the encoded value */ + private long cachedSize = -1; private T value; @@ -382,6 +384,9 @@ public WindmillValue readLater() { @Override public T read() { try (Closeable scope = scopedReadState()) { + if (!valueIsKnown) { + cachedSize = -1; + } value = getFuture().get(); valueIsKnown = true; return value; @@ -397,6 +402,7 @@ public T read() { public void write(T value) { modified = true; valueIsKnown = true; + cachedSize = -1; this.value = value; } @@ -410,14 +416,18 @@ protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKey cache) return WorkItemCommitRequest.newBuilder().buildPartial(); } - ByteString.Output stream = ByteString.newOutput(); - if (value != null) { - coder.encode(value, stream, Coder.Context.OUTER); + ByteString encoded = null; + if (cachedSize == -1 || modified) { + ByteString.Output stream = ByteString.newOutput(); + if (value != null) { + coder.encode(value, stream, Coder.Context.OUTER); + } + encoded = stream.toByteString(); + cachedSize = encoded.size(); } - ByteString encoded = stream.toByteString(); // Place in cache to avoid a future read. - cache.put(namespace, address, this, encoded.size()); + cache.put(namespace, address, this, cachedSize); if (!modified) { // The value was read, but never written or cleared. From b843bfc1a277d4cd54fb811f7eb1f622af25c8cb Mon Sep 17 00:00:00 2001 From: steve Date: Thu, 28 May 2020 13:03:03 -0400 Subject: [PATCH 108/151] [BEAM-9964] Move workerCacheMb to a user-visible place --- .../options/DataflowPipelineDebugOptions.java | 12 ++++++++++++ .../options/DataflowWorkerHarnessOptions.java | 12 ------------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index e81ab43fbb40..9a63e00ba13a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -192,6 +192,18 @@ public Dataflow create(PipelineOptions options) { void setDumpHeapOnOOM(boolean dumpHeapBeforeExit); + /** + * The size of the worker's in-memory cache, in megabytes. + * + *

    Currently, this cache is used for storing read values of side inputs. as well as the state + * for streaming jobs. + */ + @Description("The size of the worker's in-memory cache, in megabytes.") + @Default.Integer(100) + Integer getWorkerCacheMb(); + + void setWorkerCacheMb(Integer value); + /** * CAUTION: This option implies dumpHeapOnOOM, and has similar caveats. Specifically, heap dumps * can of comparable size to the default boot disk. Consider increasing the boot disk size before diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java index 3f37c74929fb..596b26dc05b8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.options; -import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Hidden; @@ -41,15 +40,4 @@ public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions { String getJobId(); void setJobId(String value); - - /** - * The size of the worker's in-memory cache, in megabytes. - * - *

    Currently, this cache is used for storing read values of side inputs. - */ - @Description("The size of the worker's in-memory cache, in megabytes.") - @Default.Integer(100) - Integer getWorkerCacheMb(); - - void setWorkerCacheMb(Integer value); } From 47ea1123c4d1bcac8333e884e713cba65d472c88 Mon Sep 17 00:00:00 2001 From: brucearctor <5032356+brucearctor@users.noreply.github.com> Date: Thu, 28 May 2020 13:05:41 -0700 Subject: [PATCH 109/151] [BEAM-9679] Flatten Kata for Go (#11806) --- .../Flatten/Flatten/cmd/main.go | 45 +++++++ .../go/Core Transforms/Flatten/Flatten/go.mod | 28 ++++ .../go/Core Transforms/Flatten/Flatten/go.sum | 123 ++++++++++++++++++ .../Flatten/Flatten/pkg/task/task.go | 22 ++++ .../Flatten/Flatten/task-info.yaml | 31 +++++ .../Core Transforms/Flatten/Flatten/task.md | 38 ++++++ .../Flatten/Flatten/test/task_test.go | 47 +++++++ .../Core Transforms/Flatten/lesson-info.yaml | 21 +++ .../go/Core Transforms/section-info.yaml | 1 + 9 files changed, 356 insertions(+) create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/cmd/main.go create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/go.mod create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/go.sum create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/pkg/task/task.go create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/task-info.yaml create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/task.md create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/test/task_test.go create mode 100644 learning/katas/go/Core Transforms/Flatten/lesson-info.yaml diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/cmd/main.go b/learning/katas/go/Core Transforms/Flatten/Flatten/cmd/main.go new file mode 100644 index 000000000000..486287241144 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/cmd/main.go @@ -0,0 +1,45 @@ +// 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 main + +import ( + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/log" + "github.com/apache/beam/sdks/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/go/pkg/beam/x/debug" + "context" + "flatten/pkg/task" +) + +func main() { + ctx := context.Background() + + p, s := beam.NewPipelineWithRoot() + + aWords := beam.Create(s, "apple", "ant", "arrow") + bWords := beam.Create(s, "ball", "book", "bow") + + + output := task.ApplyTransform(s, aWords, bWords) + + debug.Print(s, output) + + err := beamx.Run(ctx, p) + + if err != nil { + log.Exitf(context.Background(), "Failed to execute job: %v", err) + } +} diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/go.mod b/learning/katas/go/Core Transforms/Flatten/Flatten/go.mod new file mode 100644 index 000000000000..d0ed83857607 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/go.mod @@ -0,0 +1,28 @@ +// 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. + +module flatten + +go 1.13 + +require ( + github.com/apache/beam v2.20.0+incompatible + github.com/golang/protobuf v1.4.0 // indirect + github.com/google/go-cmp v0.4.0 + github.com/googleapis/gax-go v2.0.2+incompatible // indirect + go.opencensus.io v0.22.3 // indirect + google.golang.org/api v0.22.0 // indirect + google.golang.org/grpc v1.28.1 // indirect +) diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/go.sum b/learning/katas/go/Core Transforms/Flatten/Flatten/go.sum new file mode 100644 index 000000000000..15ecb7727e03 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/go.sum @@ -0,0 +1,123 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0 h1:ROfEUZz+Gh5pa62DJWXSaonyu3StP6EA6lPEXPI6mCo= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/apache/beam v2.20.0+incompatible h1:YzP/+VyAnYdu4Wjh5EkBz3vUjVdE7vUPEZ6xijCJ2sk= +github.com/apache/beam v2.20.0+incompatible/go.mod h1:/8NX3Qi8vGstDLLaeaU7+lzVEu/ACaQhYjeefzQ0y1o= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0 h1:oOuy+ugB+P/kBdUnG5QaMXSIyJ1q38wWSojYCb3z5VQ= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww= +github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45 h1:SVwTIAaPC2U/AvvLNZ2a7OVsmBpC8L5BlwK1whH3hm0= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.22.0 h1:J1Pl9P2lnmYFSJvgs70DKELqHNh8CNWXPbud4njEE2s= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.1 h1:C1QC6KzgSiLyBabDi87BbjaGreoRgGUF5nOyvfrAZ1k= +google.golang.org/grpc v1.28.1/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0 h1:qdOKuR/EIArgaWNjetjgTzgVTAZ+S/WXVrq9HW9zimw= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/pkg/task/task.go b/learning/katas/go/Core Transforms/Flatten/Flatten/pkg/task/task.go new file mode 100644 index 000000000000..bad672e42b20 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/pkg/task/task.go @@ -0,0 +1,22 @@ +// 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 task + +import "github.com/apache/beam/sdks/go/pkg/beam" + +func ApplyTransform(s beam.Scope, aInputs beam.PCollection, bInputs beam.PCollection) beam.PCollection { + return beam.Flatten(s, aInputs, bInputs) +} \ No newline at end of file diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/task-info.yaml b/learning/katas/go/Core Transforms/Flatten/Flatten/task-info.yaml new file mode 100644 index 000000000000..4f9e9b55403f --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/task-info.yaml @@ -0,0 +1,31 @@ +# 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. + +type: edu +files: +- name: go.sum + visible: false +- name: test/task_test.go + visible: false +- name: cmd/main.go + visible: true +- name: go.mod + visible: false +- name: pkg/task/task.go + visible: true + placeholders: + - offset: 972 + length: 33 + placeholder_text: TODO() diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/task.md b/learning/katas/go/Core Transforms/Flatten/Flatten/task.md new file mode 100644 index 000000000000..d1d8a42fb929 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/task.md @@ -0,0 +1,38 @@ + + +Flatten +------- + +Flatten is a Beam transform for PCollection objects that store the same data type. Flatten merges +multiple PCollection objects into a single logical PCollection. + +**Kata:** Implement a +[Flatten](https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#Flatten) +transform that merges two PCollection of words into a single PCollection. + +

    + Refer to + Flatten to solve this problem. +
    + +
    + Refer to the Beam Programming Guide + + "Flatten" section for more information. +
    \ No newline at end of file diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/test/task_test.go b/learning/katas/go/Core Transforms/Flatten/Flatten/test/task_test.go new file mode 100644 index 000000000000..2615e31f64df --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/test/task_test.go @@ -0,0 +1,47 @@ +// 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 test + +import ( + "flatten/pkg/task" + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/go/pkg/beam/testing/ptest" + "testing" +) + +func TestApplyTransform(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + tests := []struct { + aWords beam.PCollection + bWords beam.PCollection + want []interface{} + }{ + { + aWords: beam.Create(s, "apple", "ant", "arrow"), + bWords: beam.Create(s, "ball", "book", "bow"), + want: []interface{}{"apple", "ant", "arrow", "ball", "book", "bow"}, + }, + } + for _, tt := range tests { + got := task.ApplyTransform(s, tt.aWords, tt.bWords) + passert.Equals(s, got, tt.want...) + if err := ptest.Run(p); err != nil { + t.Error(err) + } + } +} + diff --git a/learning/katas/go/Core Transforms/Flatten/lesson-info.yaml b/learning/katas/go/Core Transforms/Flatten/lesson-info.yaml new file mode 100644 index 000000000000..e68515874b96 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/lesson-info.yaml @@ -0,0 +1,21 @@ +# +# 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. +# + +content: +- Flatten \ No newline at end of file diff --git a/learning/katas/go/Core Transforms/section-info.yaml b/learning/katas/go/Core Transforms/section-info.yaml index 8ea49eb04ce4..3fe207041313 100644 --- a/learning/katas/go/Core Transforms/section-info.yaml +++ b/learning/katas/go/Core Transforms/section-info.yaml @@ -20,3 +20,4 @@ content: - Map - GroupByKey +- Flatten \ No newline at end of file From 018e889829e300ab9f321da7e0010ff0011a73b1 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 28 May 2020 13:29:07 -0700 Subject: [PATCH 110/151] [BEAM-9220] Go Dataflow jobs to use runner v2 (#11207) --- sdks/go/pkg/beam/runners/dataflow/dataflow.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go index 8681ceb05e25..920731338b13 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go @@ -24,6 +24,7 @@ import ( "fmt" "io" "path" + "strings" "sync/atomic" "time" @@ -127,6 +128,17 @@ func Execute(ctx context.Context, p *beam.Pipeline) error { hooks.SerializeHooksToOptions() experiments := jobopts.GetExperiments() + // Always use runner v2, unless set already. + var v2set bool + for _, e := range experiments { + if strings.Contains(e, "use_runner_v2") || strings.Contains(e, "use_unified_worker") { + v2set = true + break + } + } + if !v2set { + experiments = append(experiments, "use_unified_worker") + } if *minCPUPlatform != "" { experiments = append(experiments, fmt.Sprintf("min_cpu_platform=%v", *minCPUPlatform)) } From d68f7f23171eecb71c4b5438f8f1be1dbc21345c Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 28 May 2020 14:19:39 -0700 Subject: [PATCH 111/151] Update multi-language roadmap status. --- .../content/en/roadmap/connectors-multi-sdk.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/website/www/site/content/en/roadmap/connectors-multi-sdk.md b/website/www/site/content/en/roadmap/connectors-multi-sdk.md index e00d6e424fe1..d632c09f0967 100644 --- a/website/www/site/content/en/roadmap/connectors-multi-sdk.md +++ b/website/www/site/content/en/roadmap/connectors-multi-sdk.md @@ -25,7 +25,7 @@ efforts. See [Beam portability framework roadmap](https://beam.apache.org/roadma # Cross-language transforms -_Last updated on November 2019._ +_Last updated on May 2020._ As an added benefit of Beam portability effort, we are able to utilize Beam transforms across SDKs. This has many benefits. @@ -33,7 +33,7 @@ As an added benefit of Beam portability effort, we are able to utilize Beam tran + Beam pipelines written using Python and Go SDKs will be able to utilize the vast selection of connectors that are currently implemented for Java SDK. + Java SDK will be able to utilize connectors for systems that only offer a Python API. + Go SDK, will be able to utilize connectors currently available for Java and Python SDKs. -* Ease of developing and maintaining Beam transforms - in general, with cross-language transforms, Beam transform authors will be able to implement new Beam transforms using a +* Ease of developing and maintaining Beam transforms - in general, with cross-language transforms, Beam transform authors will be able to implement new Beam transforms using a language of choice and utilize these transforms from other languages reducing the maintenance and support overheads. * [Beam SQL](https://beam.apache.org/documentation/dsls/sql/overview/), that is currently only available to Java SDK, will become available to Python and Go SDKs. * [Beam TFX transforms](https://www.tensorflow.org/tfx/transform/get_started), that are currently only available to Beam Python SDK pipelines will become available to Java and Go SDKs. @@ -48,9 +48,9 @@ Work related to developing/updating the cross-language transforms API for Java/P * Basic API for Java SDK - completed * Basic API for Python SDK - completed -* Basic API for Go SDK - Not started +* Basic API for Go SDK - In progress * Basic cross-language transform expansion service for Java and Python SDKs - completed -* Artifact staging - In progress - [email thread](https://lists.apache.org/thread.html/6fcee7047f53cf1c0636fb65367ef70842016d57effe2e5795c4137d@%3Cdev.beam.apache.org%3E), [doc](https://docs.google.com/document/d/1XaiNekAY2sptuQRIXpjGAyaYdSc-wlJ-VKjl04c8N48/edit#heading=h.900gc947qrw8) +* Artifact staging - mostly completed - [email thread](https://lists.apache.org/thread.html/6fcee7047f53cf1c0636fb65367ef70842016d57effe2e5795c4137d@%3Cdev.beam.apache.org%3E), [doc](https://docs.google.com/document/d/1XaiNekAY2sptuQRIXpjGAyaYdSc-wlJ-VKjl04c8N48/edit#heading=h.900gc947qrw8) ### Support for Flink runner @@ -62,27 +62,29 @@ Work related to making cross-language transforms available for Flink runner. Work related to making cross-language transforms available for Dataflow runner. -* Basic support for executing cross-language transforms on Dataflow runner +* Basic support for executing cross-language transforms on Dataflow runner + This work requires updates to Dataflow service's job submission and job execution logic. This is currently being developed at Google. ### Support for Direct runner Work related to making cross-language transforms available on Direct runner -* Basic support for executing cross-language transforms on portable Direct runner - Not started +* Basic support for executing cross-language transforms on Pyton Direct runner - completed +* Basic support for executing cross-language transforms on Java Direct runner - Not started ### Connector/transform support Ongoing and planned work related to making existing connectors/transforms available to other SDKs through the cross-language transforms framework. -* Java KafkIO - In progress - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029) +* Java KafkIO - completed - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029) * Java PubSubIO - In progress - [BEAM-7738](https://issues.apache.org/jira/browse/BEAM-7738) +* Java SQL - completed - [BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603) ### Portable Beam schema Portable Beam schema support will provide a generalized mechanism for serializing and transferring data across language boundaries which will be extremely useful for pipelines that employ cross-language transforms. -* Make row coder a standard coder and implement in python - In progress - [BEAM-7886](https://issues.apache.org/jira/browse/BEAM-7886) +* Make row coder a standard coder and implement in python - completed - [BEAM-7886](https://issues.apache.org/jira/browse/BEAM-7886) ### Integration/Performance testing From a5e642aa01f867e8c84df4a813767355eee60af4 Mon Sep 17 00:00:00 2001 From: Daniel Oliveira Date: Thu, 21 May 2020 19:55:04 -0700 Subject: [PATCH 112/151] [BEAM-9935] Respect allowed split points and fraction in Go. Adds code to more closely align with the implementations of splitting in Python and Java. Note that not all cases are implemented. There is no measurement of sub-element progress yet, nor is there sub-element (SDF) splitting yet. --- .../pkg/beam/core/runtime/exec/datasource.go | 105 +++++++++++--- .../beam/core/runtime/exec/datasource_test.go | 135 +++++++++++++++++- sdks/go/pkg/beam/core/runtime/exec/plan.go | 6 +- .../pkg/beam/core/runtime/harness/harness.go | 6 +- 4 files changed, 226 insertions(+), 26 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index ccc4a09acb50..f9e9a943cba8 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go @@ -20,6 +20,7 @@ import ( "fmt" "io" "math" + "sort" "sync" "time" @@ -266,33 +267,95 @@ func (n *DataSource) Progress() ProgressReportSnapshot { return ProgressReportSnapshot{PID: n.outputPID, ID: n.SID.PtransformID, Name: n.Name, Count: c} } -// Split takes a sorted set of potential split indices, selects and actuates -// split on an appropriate split index, and returns the selected split index -// if successful. Returns an error when unable to split. -func (n *DataSource) Split(splits []int64, frac float64) (int64, error) { - if splits == nil { - return 0, fmt.Errorf("failed to split: requested splits were empty") - } +// Split takes a sorted set of potential split indices and a fraction of the +// remainder to split at, selects and actuates a split on an appropriate split +// index, and returns the selected split index if successful or an error when +// unsuccessful. +// +// The bufSize param specifies the estimated number of elements that will be +// sent to this DataSource, and is used to be able to perform accurate splits +// even if the DataSource has not yet received all its elements. A bufSize of +// 0 or less indicates that its unknown, and so uses the current known size. +func (n *DataSource) Split(splits []int64, frac float64, bufSize int64) (int64, error) { if n == nil { return 0, fmt.Errorf("failed to split at requested splits: {%v}, DataSource not initialized", splits) } + if frac > 1.0 { + frac = 1.0 + } else if frac < 0.0 { + frac = 0.0 + } + n.mu.Lock() - c := n.index - // Find the smallest split index that we haven't yet processed, and set - // the promised split index to this value. - for _, s := range splits { - // // Never split on the first element, or the current element. - if s > 0 && s > c && s <= n.splitIdx { - n.splitIdx = s - fs := n.splitIdx - n.mu.Unlock() - return fs, nil - } + // Size to split within is the minimum of bufSize or splitIdx so we avoid + // including elements we already know won't be processed. + if bufSize <= 0 || n.splitIdx < bufSize { + bufSize = n.splitIdx + } + s, err := splitHelper(n.index, bufSize, splits, frac) + if err != nil { + n.mu.Unlock() + return 0, err } + n.splitIdx = s + fs := n.splitIdx n.mu.Unlock() - // If we can't find a suitable split index from the requested choices, - // return an error. - return 0, fmt.Errorf("failed to split at requested splits: {%v}, DataSource at index: %v", splits, c) + return fs, nil +} + +// splitHelper is a helper function that finds a split point in a range. +// currIdx and splitIdx should match the DataSource's index and splitIdx fields, +// and represent the start and end of the splittable range respectively. splits +// is an optional slice of valid split indices, and if nil then all indices are +// considered valid split points. frac must be between [0, 1], and represents +// a fraction of the remaining work that the split point aims to be as close +// as possible to. +func splitHelper(currIdx, splitIdx int64, splits []int64, frac float64) (int64, error) { + // Get split index from fraction. Find the closest index to the fraction of + // the remainder. + var start int64 = 0 + if currIdx > start { + start = currIdx + } + // This is the first valid split index, since we should never split at 0 or + // at the current element. + safeStart := start + 1 + // The remainder starts at our actual progress (i.e. start), but our final + // split index has to be >= our safeStart. + fracIdx := start + int64(math.Round(frac*float64(splitIdx-start))) + if fracIdx < safeStart { + fracIdx = safeStart + } + if splits == nil { + // All split points are valid so just split at fraction. + return fracIdx, nil + } else { + // Find the closest unprocessed split point to our fraction. + sort.Slice(splits, func(i, j int) bool { return splits[i] < splits[j] }) + var prevDiff int64 = math.MaxInt64 + var bestS int64 = -1 + for _, s := range splits { + if s >= safeStart && s <= splitIdx { + diff := intAbs(fracIdx - s) + if diff <= prevDiff { + prevDiff = diff + bestS = s + } else { + break // Stop early if the difference starts increasing. + } + } + } + if bestS != -1 { + return bestS, nil + } + } + return 0, fmt.Errorf("failed to split DataSource (at index: %v) at requested splits: {%v}", currIdx, splits) +} + +// intAbs implements absolute value for integers via Two's Complement. +func intAbs(n int64) int64 { + y := n >> 63 // y ← x ⟫ 63 + return (n ^ y) - y // (x ⨁ y) - y } type concatReStream struct { diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go b/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go index 1ce493cd1d8f..c0ff1a941ecc 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/datasource_test.go @@ -341,7 +341,7 @@ func TestDataSource_Split(t *testing.T) { <-blockedCh // Validate that we do not split on the element we're blocking on index. // The first valid split is at test.splitIdx. - if splitIdx, err := source.Split([]int64{0, 1, 2, 3, 4, 5}, -1); err != nil { + if splitIdx, err := source.Split([]int64{0, 1, 2, 3, 4, 5}, -1, 0); err != nil { t.Errorf("error in Split: %v", err) } else if got, want := splitIdx, test.splitIdx; got != want { t.Errorf("error in Split: got splitIdx = %v, want %v ", got, want) @@ -371,6 +371,57 @@ func TestDataSource_Split(t *testing.T) { } }) + // Test that the bufSize param can be used to affect the split range. + t.Run("bufSize", func(t *testing.T) { + test := struct { + splitPts []int64 + frac float64 + bufSize int64 + splitIdx int64 + expected []interface{} + }{ + // splitIdx defaults to the max int64, so if bufSize is respected + // the closest splitPt is 3, otherwise it'll be 5000. + splitPts: []int64{3, 5000}, + frac: 0.5, + bufSize: 10, + splitIdx: 3, + expected: elements[:3], + } + + source, out, pr := initSourceTest("bufSize") + p, err := NewPlan("a", []Unit{out, source}) + if err != nil { + t.Fatalf("failed to construct plan: %v", err) + } + dc := DataContext{Data: &TestDataManager{R: pr}} + ctx := context.Background() + + // StartBundle resets the source, so no splits can be actuated before then, + // which means we need to actuate the plan manually, and insert the split request + // after StartBundle. + for i, root := range p.units { + if err := root.Up(ctx); err != nil { + t.Fatalf("error in root[%d].Up: %v", i, err) + } + } + p.status = Active + + runOnRoots(ctx, t, p, "StartBundle", func(root Root, ctx context.Context) error { return root.StartBundle(ctx, "1", dc) }) + + // SDK never splits on 0, so check that every test. + sp := SplitPoints{Splits: test.splitPts, Frac: test.frac, BufSize: test.bufSize} + if splitIdx, err := p.Split(sp); err != nil { + t.Fatalf("error in Split: %v", err) + } else if got, want := splitIdx, test.splitIdx; got != want { + t.Fatalf("error in Split: got splitIdx = %v, want %v ", got, want) + } + runOnRoots(ctx, t, p, "Process", Root.Process) + runOnRoots(ctx, t, p, "FinishBundle", Root.FinishBundle) + + validateSource(t, out, source, makeValues(test.expected...)) + }) + // Test expects splitting errors, but for processing to be successful. t.Run("errors", func(t *testing.T) { source, out, pr := initSourceTest("noSplitsUntilStarted") @@ -410,15 +461,93 @@ func TestDataSource_Split(t *testing.T) { t.Run("sanity_errors", func(t *testing.T) { var source *DataSource - if _, err := source.Split([]int64{0}, -1); err == nil { + if _, err := source.Split([]int64{0}, -1, 0); err == nil { t.Fatal("expected error splitting nil *DataSource") } - if _, err := source.Split(nil, -1); err == nil { + if _, err := source.Split(nil, -1, 0); err == nil { t.Fatal("expected error splitting nil desired splits") } }) } +// TestSplitHelper tests the underlying split logic to confirm that various +// cases produce expected split points. +func TestSplitHelper(t *testing.T) { + // Test splits at various fractions. + t.Run("SimpleSplits", func(t *testing.T) { + tests := []struct { + curr, size int64 + frac float64 + want int64 + }{ + // Split as close to the beginning as possible. + {curr: 0, size: 16, frac: 0, want: 1}, + // The closest split is at 4, even when just above or below it. + {curr: 0, size: 16, frac: 0.24, want: 4}, + {curr: 0, size: 16, frac: 0.25, want: 4}, + {curr: 0, size: 16, frac: 0.26, want: 4}, + // Split the *remainder* in half. + {curr: 0, size: 16, frac: 0.5, want: 8}, + {curr: 2, size: 16, frac: 0.5, want: 9}, + {curr: 6, size: 16, frac: 0.5, want: 11}, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(%v of [%v, %v])", test.frac, test.curr, test.size), func(t *testing.T) { + got, err := splitHelper(test.curr, test.size, nil, test.frac) + if err != nil { + t.Errorf("error in splitHelper: %v", err) + } else if got != test.want { + t.Errorf("incorrect split point: got: %v, want: %v", got, test.want) + } + }) + } + }) + + // Test splits with allowed split points. + t.Run("WithAllowedSplits", func(t *testing.T) { + tests := []struct { + curr, size int64 + splits []int64 + frac float64 + want int64 + err bool // True if test should cause a failure. + }{ + // The desired split point is at 4. + {curr: 0, size: 16, splits: []int64{2, 3, 4, 5}, frac: 0.25, want: 4}, + // If we can't split at 4, choose the closest possible split point. + {curr: 0, size: 16, splits: []int64{2, 3, 5}, frac: 0.25, want: 5}, + {curr: 0, size: 16, splits: []int64{2, 3, 6}, frac: 0.25, want: 3}, + // Also test the case where all possible split points lie above or + // below the desired split point. + {curr: 0, size: 16, splits: []int64{5, 6, 7}, frac: 0.25, want: 5}, + {curr: 0, size: 16, splits: []int64{1, 2, 3}, frac: 0.25, want: 3}, + // We have progressed beyond all possible split points, so can't split. + {curr: 5, size: 16, splits: []int64{1, 2, 3}, frac: 0.25, err: true}, + } + for _, test := range tests { + test := test + t.Run(fmt.Sprintf("(%v of [%v, %v], splits = %v)", test.frac, test.curr, test.size, test.splits), func(t *testing.T) { + got, err := splitHelper(test.curr, test.size, test.splits, test.frac) + if test.err { + if err == nil { + t.Errorf("splitHelper should have errored, instead got: %v", got) + } + } else { + if err != nil { + t.Errorf("error in splitHelper: %v", err) + } else if got != test.want { + t.Errorf("incorrect split point: got: %v, want: %v", got, test.want) + } + } + }) + } + }) + + // TODO(BEAM-9935): Add SDF and element progress splitting tests from Java + // and Python once those features are added. +} + func runOnRoots(ctx context.Context, t *testing.T, p *Plan, name string, mthd func(Root, context.Context) error) { t.Helper() for i, root := range p.roots { diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go index fde9e7c78fa0..2f07f3c3dbaf 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/plan.go +++ b/sdks/go/pkg/beam/core/runtime/exec/plan.go @@ -198,6 +198,10 @@ type SplitPoints struct { // Splits is a list of desired split indices. Splits []int64 Frac float64 + + // Estimated total number of elements (including unsent) for the source. + // A zero value indicates unknown, instead use locally known size. + BufSize int64 } // Split takes a set of potential split indexes, and if successful returns @@ -206,7 +210,7 @@ type SplitPoints struct { // Returns an error when unable to split. func (p *Plan) Split(s SplitPoints) (int64, error) { if p.source != nil { - return p.source.Split(s.Splits, s.Frac) + return p.source.Split(s.Splits, s.Frac, s.BufSize) } return 0, fmt.Errorf("failed to split at requested splits: {%v}, Source not initialized", s) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index 26cd02e7b036..44099210d049 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -310,7 +310,11 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe if ds == nil { return fail(ctx, instID, "failed to split: desired splits for root of %v was empty.", ref) } - split, err := plan.Split(exec.SplitPoints{Splits: ds.GetAllowedSplitPoints(), Frac: ds.GetFractionOfRemainder()}) + split, err := plan.Split(exec.SplitPoints{ + Splits: ds.GetAllowedSplitPoints(), + Frac: ds.GetFractionOfRemainder(), + BufSize: ds.GetEstimatedInputElements(), + }) if err != nil { return fail(ctx, instID, "unable to split %v: %v", ref, err) From 775ad8edbb95244b03e56e2bd0bbf20b11eba22d Mon Sep 17 00:00:00 2001 From: Udi Meiri Date: Thu, 28 May 2020 15:14:36 -0700 Subject: [PATCH 113/151] [BEAM-8280] Blog post: Python typing changes (#11070) * [BEAM-8280] Blog post: Python typing changes Co-authored-by: Chad Dombrova * Added excerpt and reworked sidebar. * Rename file to match new style and update headers More cleanup * More fixes, blog with dates in the future don't appear so I changed the date today. * Also update link to blog post * Update date to today Co-authored-by: Chad Dombrova --- CHANGES.md | 2 +- .../www/site/content/en/blog/python-typing.md | 136 ++++++++++++++++++ website/www/site/data/authors.yml | 3 + 3 files changed, 140 insertions(+), 1 deletion(-) create mode 100644 website/www/site/content/en/blog/python-typing.md diff --git a/CHANGES.md b/CHANGES.md index affe1e068786..d3e706515ae5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -108,7 +108,7 @@ for example usage. More details will be in [Ensuring Python Type Safety](https://beam.apache.org/documentation/sdks/python-type-safety/) and an upcoming - [blog post](https://beam.apache.org/blog/python/typing/2020/03/06/python-typing.html). + [blog post](https://beam.apache.org/blog/python-typing/index.html). * Java SDK: Introducing the concept of options in Beam Schema’s. These options add extra context to fields and schemas. This replaces the current Beam metadata that is present diff --git a/website/www/site/content/en/blog/python-typing.md b/website/www/site/content/en/blog/python-typing.md new file mode 100644 index 000000000000..7982c19fe9e4 --- /dev/null +++ b/website/www/site/content/en/blog/python-typing.md @@ -0,0 +1,136 @@ +--- +layout: post +title: "Python SDK Typing Changes" +date: 2020-05-28 00:00:01 -0800 +categories: + - blog + - python + - typing +authors: + - chadrik + - udim +--- + + +Beam Python has recently increased its support and integration of Python 3 type +annotations for improved code clarity and type correctness checks. +Read on to find out what's new. + + + +Python supports type annotations on functions (PEP 484). Static type checkers, +such as mypy, are used to verify adherence to these types. +For example: +``` +def f(v: int) -> int: + return v[0] +``` +Running mypy on the above code will give the error: +`Value of type "int" is not indexable`. + +We've recently made changes to Beam in 2 areas: + +Adding type annotations throughout Beam. Type annotations make a large and +sophisticated codebase like Beam easier to comprehend and navigate in your +favorite IDE. + +Second, we've added support for Python 3 type annotations. This allows SDK +users to specify a DoFn's type hints in one place. +We've also expanded Beam's support of `typing` module types. + +For more background see: +[Ensuring Python Type Safety](https://beam.apache.org/documentation/sdks/python-type-safety/). + +# Beam Is Typed + +In tandem with the new type annotation support within DoFns, we've invested a +great deal of time adding type annotations to the Beam python code itself. +With this in place, we have begun using mypy, a static type +checker, as part of Beam's code review process, which ensures higher quality +contributions and fewer bugs. +The added context and insight that type annotations add throughout Beam is +useful for all Beam developers, contributors and end users alike, but +it is especially beneficial for developers who are new to the project. +If you use an IDE that understands type annotations, it will provide richer +type completions and warnings than before. +You'll also be able to use your IDE to inspect the types of Beam functions and +transforms to better understand how they work, which will ease your own +development. +Finally, once Beam is fully annotated, end users will be able to benefit from +the use of static type analysis on their own pipelines and custom transforms. + +# New Ways to Annotate + +## Python 3 Syntax Annotations + +Coming in Beam 2.21 (BEAM-8280), you will be able to use Python annotation +syntax to specify input and output types. + +For example, this new form: +``` +class MyDoFn(beam.DoFn): + def process(self, element: int) -> typing.Text: + yield str(element) +``` +is equivalent to this: +``` +@apache_beam.typehints.with_input_types(int) +@apache_beam.typehints.with_output_types(typing.Text) +class MyDoFn(beam.DoFn): + def process(self, element): + yield str(element) +``` + +One of the advantages of the new form is that you may already be using it +in tandem with a static type checker such as mypy, thus getting additional +runtime type checking for free. + +This feature will be enabled by default, and there will be 2 mechanisms in +place to disable it: +1. Calling `apache_beam.typehints.disable_type_annotations()` before pipeline +construction will disable the new feature completely. +1. Decorating a function with `@apache_beam.typehints.no_annotations` will +tell Beam to ignore annotations for it. + +Uses of Beam's `with_input_type`, `with_output_type` methods and decorators will +still work and take precedence over annotations. + +### Sidebar + +You might ask: couldn't we use mypy to type check Beam pipelines? +There are several reasons why this is not the case. +- Pipelines are constructed at runtime and may depend on information that is +only known at that time, such as a config file or database table schema. +- PCollections don't have the necessary type information, so mypy sees them as +effectively containing any element type. +This may change in in the future. +- Transforms using lambdas (ex: `beam.Map(lambda x: (1, x)`) cannot be +annotated properly using PEP 484. +However, Beam does a best-effort attempt to analyze the output type +from the bytecode. + +## Typing Module Support + +Python's [typing](https://docs.python.org/3/library/typing.html) module defines +types used in type annotations. This is what we call "native" types. +While Beam has its own typing types, it also supports native types. +While both Beam and native types are supported, for new code we encourage using +native typing types. Native types have as these are supported by additional tools. + +While working on Python 3 annotations syntax support, we've also discovered and +fixed issues with native type support. There may still be bugs and unsupported +native types. Please +[let us know](https://beam.apache.org/community/contact-us/) if you encounter +issues. diff --git a/website/www/site/data/authors.yml b/website/www/site/data/authors.yml index 8a111a688cec..376cfdbaa62b 100644 --- a/website/www/site/data/authors.yml +++ b/website/www/site/data/authors.yml @@ -30,6 +30,9 @@ anton: ccy: name: Charles Chen email: ccy@apache.org +chadrik: + name: Chad Dombrova + email: chadrik@apache.org chamikara: name: Chamikara Jayalath email: chamikara@apache.org From e7abbdbf7ebd56c14b7ecf4831b70bf08218d0b3 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 28 May 2020 15:58:01 -0700 Subject: [PATCH 114/151] Update website/www/site/content/en/roadmap/connectors-multi-sdk.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ismaël Mejía --- website/www/site/content/en/roadmap/connectors-multi-sdk.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/www/site/content/en/roadmap/connectors-multi-sdk.md b/website/www/site/content/en/roadmap/connectors-multi-sdk.md index d632c09f0967..55d11fd73d32 100644 --- a/website/www/site/content/en/roadmap/connectors-multi-sdk.md +++ b/website/www/site/content/en/roadmap/connectors-multi-sdk.md @@ -76,7 +76,7 @@ Work related to making cross-language transforms available on Direct runner Ongoing and planned work related to making existing connectors/transforms available to other SDKs through the cross-language transforms framework. -* Java KafkIO - completed - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029) +* Java KafkaIO - completed - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029) * Java PubSubIO - In progress - [BEAM-7738](https://issues.apache.org/jira/browse/BEAM-7738) * Java SQL - completed - [BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603) @@ -96,4 +96,3 @@ Work related to adding documenting on cross-language transforms to Beam Website. * Document cross-language transforms API for Java/Python - Not started * Document API for making existing transforms available as cross-language transforms for Java/Python - Not started - From 9e5dd68bb951cf8c7faaf15d4e5852b0fe6963d2 Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Fri, 22 May 2020 02:13:45 -0700 Subject: [PATCH 115/151] [BEAM-10125] adding cross-language KafkaIO integration test --- .../io/external/xlang_kafkaio_it_test.py | 141 ++++++++++++++++++ 1 file changed, 141 insertions(+) create mode 100644 sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py diff --git a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py new file mode 100644 index 000000000000..7dbbf837748d --- /dev/null +++ b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py @@ -0,0 +1,141 @@ +# +# 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. +# + +"""Integration test for Python cross-language pipelines for Java KafkaIO.""" + +from __future__ import absolute_import + +import contextlib +import logging +import os +import socket +import subprocess +import time +import typing +import unittest + +import apache_beam as beam +from apache_beam.io.external.kafka import ReadFromKafka +from apache_beam.io.external.kafka import WriteToKafka +from apache_beam.metrics import Metrics +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.testing.test_pipeline import TestPipeline + + +class CrossLanguageKafkaIO(object): + def __init__(self, bootstrap_servers, topic, expansion_service=None): + self.bootstrap_servers = bootstrap_servers + self.topic = topic + self.expansion_service = expansion_service + self.sum_counter = Metrics.counter('source', 'elements_sum') + + def build_write_pipeline(self, pipeline): + _ = ( + pipeline + | 'Impulse' >> beam.Impulse() + | 'Generate' >> beam.FlatMap(lambda x: range(1000)) # pylint: disable=range-builtin-not-iterating + | 'Reshuffle' >> beam.Reshuffle() + | 'MakeKV' >> beam.Map(lambda x: + (b'', str(x).encode())).with_output_types( + typing.Tuple[bytes, bytes]) + | 'WriteToKafka' >> WriteToKafka( + producer_config={'bootstrap.servers': self.bootstrap_servers}, + topic=self.topic, + expansion_service=self.expansion_service)) + + def build_read_pipeline(self, pipeline): + _ = ( + pipeline + | 'ReadFromKafka' >> ReadFromKafka( + consumer_config={ + 'bootstrap.servers': self.bootstrap_servers, + 'auto.offset.reset': 'earliest' + }, + topics=[self.topic], + expansion_service=self.expansion_service) + | 'Windowing' >> beam.WindowInto( + beam.window.FixedWindows(300), + trigger=beam.transforms.trigger.AfterProcessingTime(60), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING) + | 'DecodingValue' >> beam.Map(lambda elem: int(elem[1].decode())) + | 'CombineGlobally' >> beam.CombineGlobally(sum).without_defaults() + | 'SetSumCounter' >> beam.Map(self.sum_counter.inc)) + + def run_xlang_kafkaio(self, pipeline): + self.build_write_pipeline(pipeline) + self.build_read_pipeline(pipeline) + pipeline.run(False) + + +@unittest.skipUnless( + os.environ.get('LOCAL_KAFKA_JAR'), + "LOCAL_KAFKA_JAR environment var is not provided.") +class CrossLanguageKafkaIOTest(unittest.TestCase): + def get_open_port(self): + s = None + try: + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + except: # pylint: disable=bare-except + # Above call will fail for nodes that only support IPv6. + s = socket.socket(socket.AF_INET6, socket.SOCK_STREAM) + s.bind(('localhost', 0)) + s.listen(1) + port = s.getsockname()[1] + s.close() + return port + + @contextlib.contextmanager + def local_kafka_service(self, local_kafka_jar_file): + kafka_port = str(self.get_open_port()) + zookeeper_port = str(self.get_open_port()) + kafka_server = None + try: + kafka_server = subprocess.Popen( + ['java', '-jar', local_kafka_jar_file, kafka_port, zookeeper_port]) + time.sleep(3) + yield kafka_port + finally: + if kafka_server: + kafka_server.kill() + + def get_options(self): + options = PipelineOptions([ + '--runner', + 'FlinkRunner', + '--parallelism', + '2', + '--experiment', + 'beam_fn_api' + ]) + return options + + def test_kafkaio_write(self): + local_kafka_jar = os.environ.get('LOCAL_KAFKA_JAR') + with self.local_kafka_service(local_kafka_jar) as kafka_port: + options = self.get_options() + p = TestPipeline(options=options) + p.not_use_test_runner_api = True + CrossLanguageKafkaIO('localhost:%s' % kafka_port, + 'xlang_kafkaio_test').build_write_pipeline(p) + job = p.run() + job.wait_until_finish() + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From 6dc08a4fb361b19a3353e9bd93563bc1120723df Mon Sep 17 00:00:00 2001 From: David Moravek Date: Fri, 29 May 2020 17:41:32 +0200 Subject: [PATCH 116/151] [BEAM-8608] Chain DoFns in Flink batch runner when possible. --- .../beam/runners/flink/FlinkCapabilities.java | 34 ++++ .../beam/runners/flink/FlinkCapabilities.java | 34 ++++ .../flink/FlinkBatchTransformTranslators.java | 21 ++- .../functions/FlinkDoFnFunction.java | 156 ++++++++++++------ .../functions/FlinkStatefulDoFnFunction.java | 7 +- 5 files changed, 193 insertions(+), 59 deletions(-) create mode 100644 runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java create mode 100644 runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java diff --git a/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java b/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java new file mode 100644 index 000000000000..aabdc977e995 --- /dev/null +++ b/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java @@ -0,0 +1,34 @@ +/* + * 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.beam.runners.flink; + +/** Handle different capabilities between flink versions. */ +public class FlinkCapabilities { + + /** + * Support for outputting elements in close method of chained drivers. + * + *

    {@see } for more + * details. + * + * @return True if feature is supported. + */ + public static boolean supportsOutputInTearDown() { + return true; + } +} diff --git a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java new file mode 100644 index 000000000000..02586e38a262 --- /dev/null +++ b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java @@ -0,0 +1,34 @@ +/* + * 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.beam.runners.flink; + +/** Handle different capabilities between flink versions. */ +public class FlinkCapabilities { + + /** + * Support for outputting elements in close method of chained drivers. + * + *

    {@see } for more + * details. + * + * @return True if feature is supported. + */ + public static boolean supportsOutputInTearDown() { + return false; + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 4254a0604c78..9ddf5049dc01 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -240,7 +240,7 @@ public void translateNode( PTransform>, PCollection>>> transform, FlinkBatchTranslationContext context) { - // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API + // for now, this is copied from the Combine.PerKey translator. Once we have the new runner API // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn DataSet>> inputDataSet = @@ -527,11 +527,12 @@ public void translateNode( Map, PValue> outputs = context.getOutputs(transform); - TupleTag mainOutputTag; + final TupleTag mainOutputTag; DoFnSchemaInformation doFnSchemaInformation; Map> sideInputMapping; try { - mainOutputTag = ParDoTranslation.getMainOutputTag(context.getCurrentTransform()); + mainOutputTag = + (TupleTag) ParDoTranslation.getMainOutputTag(context.getCurrentTransform()); } catch (IOException e) { throw new RuntimeException(e); } @@ -638,8 +639,8 @@ public void translateNode( outputDataSet = new GroupReduceOperator(grouping, typeInformation, doFnWrapper, fullName); } else { - FlinkDoFnFunction doFnWrapper = - new FlinkDoFnFunction( + final FlinkDoFnFunction doFnWrapper = + new FlinkDoFnFunction<>( doFn, fullName, windowingStrategy, @@ -652,8 +653,14 @@ public void translateNode( doFnSchemaInformation, sideInputMapping); - outputDataSet = - new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, fullName); + if (FlinkCapabilities.supportsOutputInTearDown()) { + outputDataSet = + new FlatMapOperator<>(inputDataSet, typeInformation, doFnWrapper, fullName); + } else { + // This can be removed once we drop support for 1.9 and 1.10 versions. + outputDataSet = + new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, fullName); + } } transformSideInputs(sideInputs, outputDataSet, context); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index a34d840e7417..7a77c8d8ddfe 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -19,7 +19,9 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; +import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.construction.SerializablePipelineOptions; @@ -40,7 +42,9 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; -import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; @@ -53,8 +57,9 @@ * all outputs with the output number. Afterwards a filter will filter out those elements that are * not to be in a specific output. */ -public class FlinkDoFnFunction - extends RichMapPartitionFunction, WindowedValue> { +public class FlinkDoFnFunction extends AbstractRichFunction + implements FlatMapFunction, WindowedValue>, + MapPartitionFunction, WindowedValue> { private final SerializablePipelineOptions serializedOptions; @@ -71,9 +76,14 @@ public class FlinkDoFnFunction private final DoFnSchemaInformation doFnSchemaInformation; private final Map> sideInputMapping; + private transient CollectorAware collectorAware; private transient DoFnInvoker doFnInvoker; + private transient DoFnRunner doFnRunner; private transient FlinkMetricContainer metricContainer; + private boolean bundleStarted = false; + private boolean exceptionThrownInFlatMap = false; + public FlinkDoFnFunction( DoFn doFn, String stepName, @@ -100,22 +110,49 @@ public FlinkDoFnFunction( this.sideInputMapping = sideInputMapping; } + @Override + public void flatMap(WindowedValue value, Collector> out) { + try { + if (!bundleStarted) { + bundleStarted = true; + doFnRunner.startBundle(); + } + collectorAware.setCollector(out); + doFnRunner.processElement(value); + } catch (Exception e) { + exceptionThrownInFlatMap = true; + throw e; + } + } + @Override public void mapPartition( - Iterable> values, Collector> out) - throws Exception { + Iterable> values, Collector> out) { + for (WindowedValue value : values) { + flatMap(value, out); + } + } - RuntimeContext runtimeContext = getRuntimeContext(); + @Override + public void open(Configuration parameters) { + // Note that the SerializablePipelineOptions already initialize FileSystems in the readObject() + // deserialization method. However, this is a hack, and we want to properly initialize the + // options where they are needed. + FileSystems.setDefaultPipelineOptions(serializedOptions.get()); + doFnInvoker = DoFnInvokers.tryInvokeSetupFor(doFn); + metricContainer = new FlinkMetricContainer(getRuntimeContext()); - DoFnRunners.OutputManager outputManager; + // setup DoFnRunner + final RuntimeContext runtimeContext = getRuntimeContext(); + final DoFnRunners.OutputManager outputManager; if (outputMap.size() == 1) { - outputManager = new FlinkDoFnFunction.DoFnOutputManager(out); + outputManager = new DoFnOutputManager(); } else { // it has some additional outputs - outputManager = new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap); + outputManager = new MultiDoFnOutputManager(outputMap); } - List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); + final List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); DoFnRunner doFnRunner = DoFnRunners.simpleRunner( @@ -132,79 +169,100 @@ public void mapPartition( doFnSchemaInformation, sideInputMapping); - FlinkPipelineOptions pipelineOptions = serializedOptions.get().as(FlinkPipelineOptions.class); - if (!pipelineOptions.getDisableMetrics()) { + if (!serializedOptions.get().as(FlinkPipelineOptions.class).getDisableMetrics()) { doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, metricContainer); } - doFnRunner.startBundle(); - - for (WindowedValue value : values) { - doFnRunner.processElement(value); - } - - doFnRunner.finishBundle(); - } - - @Override - public void open(Configuration parameters) { - // Note that the SerializablePipelineOptions already initialize FileSystems in the readObject() - // deserialization method. However, this is a hack, and we want to properly initialize the - // options where they are needed. - FileSystems.setDefaultPipelineOptions(serializedOptions.get()); - doFnInvoker = DoFnInvokers.tryInvokeSetupFor(doFn); - metricContainer = new FlinkMetricContainer(getRuntimeContext()); + this.collectorAware = (CollectorAware) outputManager; + this.doFnRunner = doFnRunner; } @Override public void close() throws Exception { + Exception suppressed = null; + try { + if (bundleStarted && !exceptionThrownInFlatMap) { + doFnRunner.finishBundle(); + } + } catch (Exception e) { + // Suppress exception, so we can properly teardown DoFn. + suppressed = e; + } try { metricContainer.registerMetricsForPipelineResult(); Optional.ofNullable(doFnInvoker).ifPresent(DoFnInvoker::invokeTeardown); + if (suppressed != null) { + throw suppressed; + } } finally { Workarounds.deleteStaticCaches(); } } - static class DoFnOutputManager implements DoFnRunners.OutputManager { + interface CollectorAware { + + void setCollector(Collector> collector); + } + + static class DoFnOutputManager implements DoFnRunners.OutputManager, CollectorAware { + + @Nullable private Collector> collector; - private Collector collector; + DoFnOutputManager() { + this(null); + } - DoFnOutputManager(Collector collector) { + DoFnOutputManager(@Nullable Collector> collector) { this.collector = collector; } @Override - @SuppressWarnings("unchecked") + public void setCollector(Collector> collector) { + this.collector = Objects.requireNonNull(collector); + } + + @Override public void output(TupleTag tag, WindowedValue output) { - collector.collect( - WindowedValue.of( - new RawUnionValue(0 /* single output */, output.getValue()), - output.getTimestamp(), - output.getWindows(), - output.getPane())); + Objects.requireNonNull(collector) + .collect( + WindowedValue.of( + new RawUnionValue(0 /* single output */, output.getValue()), + output.getTimestamp(), + output.getWindows(), + output.getPane())); } } - static class MultiDoFnOutputManager implements DoFnRunners.OutputManager { + static class MultiDoFnOutputManager implements DoFnRunners.OutputManager, CollectorAware { - private Collector> collector; - private Map, Integer> outputMap; + @Nullable private Collector> collector; + private final Map, Integer> outputMap; + + MultiDoFnOutputManager(Map, Integer> outputMap) { + this.outputMap = outputMap; + } MultiDoFnOutputManager( - Collector> collector, Map, Integer> outputMap) { + @Nullable Collector> collector, + Map, Integer> outputMap) { this.collector = collector; this.outputMap = outputMap; } + @Override + public void setCollector(Collector> collector) { + this.collector = Objects.requireNonNull(collector); + } + @Override public void output(TupleTag tag, WindowedValue output) { - collector.collect( - WindowedValue.of( - new RawUnionValue(outputMap.get(tag), output.getValue()), - output.getTimestamp(), - output.getWindows(), - output.getPane())); + Objects.requireNonNull(collector) + .collect( + WindowedValue.of( + new RawUnionValue(outputMap.get(tag), output.getValue()), + output.getTimestamp(), + output.getWindows(), + output.getPane())); } } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 1dc13fe2686d..fdc0df297ddf 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -42,6 +42,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -59,7 +60,7 @@ /** A {@link RichGroupReduceFunction} for stateful {@link ParDo} in Flink Batch Runner. */ public class FlinkStatefulDoFnFunction - extends RichGroupReduceFunction>, WindowedValue> { + extends RichGroupReduceFunction>, WindowedValue> { private final DoFn, OutputT> dofn; private String stepName; @@ -104,7 +105,7 @@ public FlinkStatefulDoFnFunction( @Override public void reduce( - Iterable>> values, Collector> out) + Iterable>> values, Collector> out) throws Exception { RuntimeContext runtimeContext = getRuntimeContext(); @@ -113,7 +114,7 @@ public void reduce( outputManager = new FlinkDoFnFunction.DoFnOutputManager(out); } else { // it has some additional Outputs - outputManager = new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap); + outputManager = new FlinkDoFnFunction.MultiDoFnOutputManager(out, outputMap); } final Iterator>> iterator = values.iterator(); From 7978c0fe04385358c6816b79d6a2fcda9a858df0 Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Fri, 29 May 2020 08:49:15 -0700 Subject: [PATCH 117/151] [BEAM-10078] uniquify Dataflow specific jars when staging (#11814) * [BEAM-10078] uniquify Dataflow specific jars when staging * add comments and logging --- .../runners/dataflow/util/PackageUtil.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index 72139931edfd..036d2dbf059f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -397,10 +397,27 @@ public static PackageAttributes forFileToStage( String.format("Non-existent file to stage: %s", file.getAbsolutePath())); } checkState(!file.isDirectory(), "Source file must not be a directory."); + String target; + // Dataflow worker jar and windmill binary can be overridden by providing files with + // predefined file names. Normally, we can use the artifact file name as same as + // the last component of GCS object resource path. However, we need special handling + // for those predefined names since they also need to be unique even in the same + // staging directory. + switch (dest) { + case "dataflow-worker.jar": + case "windmill_main": + target = + Environments.createStagingFileName( + file, Files.asByteSource(file).hash(Hashing.sha256())); + LOG.info("Staging custom {} as {}", dest, target); + break; + default: + target = dest; + } DataflowPackage destination = new DataflowPackage(); String resourcePath = FileSystems.matchNewResource(stagingPath, true) - .resolve(dest, StandardResolveOptions.RESOLVE_FILE) + .resolve(target, StandardResolveOptions.RESOLVE_FILE) .toString(); destination.setLocation(resourcePath); destination.setName(dest); From fc941a02fd2ad1e77dd4db2ce4c298bbc2a206b6 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Fri, 29 May 2020 12:50:48 -0400 Subject: [PATCH 118/151] =?UTF-8?q?[BEAM-10107]=20Remove=20outdated=20inst?= =?UTF-8?q?ructions=20for=20website=20updates=20in=20rele=E2=80=A6=20(#118?= =?UTF-8?q?52)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [BEAM-10107] Remove outdated instructions for website updates in release guide. * Replace outdated example PR. * Add example PR for beam-site and remove redundant instructions. --- .../content/en/contribute/release-guide.md | 31 +++++-------------- 1 file changed, 8 insertions(+), 23 deletions(-) diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md index 1ecc1368e801..441a2167901e 100644 --- a/website/www/site/content/en/contribute/release-guide.md +++ b/website/www/site/content/en/contribute/release-guide.md @@ -618,10 +618,6 @@ For this step, we recommend you using automation script to create a RC, but you - Please follow the [user guide](https://github.com/apache/beam-wheels#user-guide) to build python wheels. - Once all python wheels have been staged to GCS, please run [./sign_hash_python_wheels.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/sign_hash_python_wheels.sh), which copies the wheels along with signatures and hashes to [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/). - 1. Update Beam website ([example](https://github.com/apache/beam/pull/11727)) - 1. Update release version in `website/www/site/config.toml`. - 1. Add new release in `website/www/site/content/en/get-started/downloads.md`. - 1. Update `website/www/site/static/.htaccess` to redirect to the new version. ********** @@ -674,26 +670,17 @@ to the Beam website, usually within an hour. **PR 1: apache/beam-site** This pull request is against the `apache/beam-site` repo, on the `release-docs` -branch. - -* Add the new Javadoc to [SDK API Reference page](https://beam.apache.org/releases/javadoc/) page, as follows: - * Unpack the Maven artifact `org.apache.beam:beam-sdks-java-javadoc` into some temporary location. Call this `${JAVADOC_TMP}`. - * Copy the generated Javadoc into the website repository: `cp -r ${JAVADOC_TMP} javadoc/${RELEASE}`. -* Add the new Pydoc to [SDK API Reference page](https://beam.apache.org/releases/pydoc/) page, as follows: - * Copy the generated Pydoc into the website repository: `cp -r ${PYDOC_ROOT} pydoc/${RELEASE}`. - * Remove `.doctrees` directory. -* Stage files using: `git add --all javadoc/ pydoc/`. +branch ([example](https://github.com/apache/beam-site/pull/603)). +It is created by `build_release_candidate.sh` (see above). **PR 2: apache/beam** -This pull request is against the `apache/beam` repo, on the `master` branch. +This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/11727)). -* Update the `release_latest` version flag in `/website/_config.yml`, and list - the new release in `/website/src/get-started/downloads.md`, linking to the - source code download and the Release Notes in JIRA. -* Update the `RedirectMatch` rule in - [/website/src/.htaccess](https://github.com/apache/beam/blob/master/website/src/.htaccess) - to point to the new release. See file history for examples. +* Update release version in `website/www/site/config.toml`. +* Add new release in `website/www/site/content/en/get-started/downloads.md`. + * Download links will not work until the release is finalized. +* Update `website/www/site/static/.htaccess` to redirect to the new version. ### Blog post @@ -1141,12 +1128,10 @@ All wheels should be published, in addition to the zip of the release source. Copy the source release from the `dev` repository to the `release` repository at `dist.apache.org` using Subversion. -Move last release artifacts from `dist.apache.org` to `archive.apache.org` using Subversion. Then update download address for last release version, [example PR](https://github.com/apache/beam-site/pull/478). +Move last release artifacts from `dist.apache.org` to `archive.apache.org` using Subversion. Make sure to change these links on the website ([example](https://github.com/apache/beam/pull/11727)). __NOTE__: Only PMC members have permissions to do it, ping [dev@](mailto:dev@beam.apache.org) for assitance; -Make sure the download address for last release version is upldaed, [example PR](https://github.com/apache/beam-site/pull/478). - ### Deploy SDK docker images to DockerHub * Script: [publish_docker_images.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/publish_docker_images.sh) * Usage From 193e54c3e2f0fffa8d92dfed998800ea86cec073 Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Fri, 29 May 2020 10:53:57 -0700 Subject: [PATCH 119/151] [BEAM-9679] Add a CoGroupByKey lesson to the Core Transforms section (#11803) * Add a CoGroupByKey lesson to the Core Transforms section --- .../CoGroupByKey/CoGroupByKey/cmd/main.go | 44 +++ .../CoGroupByKey/CoGroupByKey/go.mod | 26 ++ .../CoGroupByKey/CoGroupByKey/go.sum | 314 ++++++++++++++++++ .../CoGroupByKey/pkg/task/task.go | 52 +++ .../CoGroupByKey/CoGroupByKey/task-info.yaml | 35 ++ .../CoGroupByKey/task-remote-info.yaml | 2 + .../CoGroupByKey/CoGroupByKey/task.md | 104 ++++++ .../CoGroupByKey/test/task_test.go | 55 +++ .../CoGroupByKey/lesson-info.yaml | 21 ++ .../CoGroupByKey/lesson-remote-info.yaml | 3 + .../Flatten/Flatten/task-remote-info.yaml | 2 + .../Flatten/lesson-remote-info.yaml | 3 + .../Map/ParDo OneToMany/go.mod | 3 +- .../Map/ParDo OneToMany/go.sum | 5 +- .../Map/ParDo OneToMany/task-remote-info.yaml | 2 +- .../Core Transforms/Map/ParDo Struct/go.mod | 3 +- .../Core Transforms/Map/ParDo Struct/go.sum | 5 +- .../Map/ParDo Struct/task-remote-info.yaml | 2 +- .../katas/go/Core Transforms/Map/ParDo/go.mod | 3 +- .../katas/go/Core Transforms/Map/ParDo/go.sum | 5 +- .../Map/ParDo/task-remote-info.yaml | 2 +- .../go/Core Transforms/section-info.yaml | 3 +- .../Hello Beam/Hello Beam Test/go.mod | 2 +- .../Hello Beam Test/task-remote-info.yaml | 2 +- .../Introduction/Hello Beam/Hello Beam/go.mod | 2 +- .../Hello Beam/task-remote-info.yaml | 2 +- learning/katas/go/course-remote-info.yaml | 2 +- 27 files changed, 683 insertions(+), 21 deletions(-) create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/cmd/main.go create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.mod create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.sum create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/pkg/task/task.go create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-info.yaml create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-remote-info.yaml create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task.md create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/test/task_test.go create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/lesson-info.yaml create mode 100644 learning/katas/go/Core Transforms/CoGroupByKey/lesson-remote-info.yaml create mode 100644 learning/katas/go/Core Transforms/Flatten/Flatten/task-remote-info.yaml create mode 100644 learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/cmd/main.go b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/cmd/main.go new file mode 100644 index 000000000000..ee11b3b65a5e --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/cmd/main.go @@ -0,0 +1,44 @@ +// 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 main + +import ( + "cogroupbykey/pkg/task" + "context" + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/log" + "github.com/apache/beam/sdks/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/go/pkg/beam/x/debug" +) + +func main() { + ctx := context.Background() + + p, s := beam.NewPipelineWithRoot() + + fruits := beam.Create(s.Scope("Fruits"), "apple", "banana", "cherry") + countries := beam.Create(s.Scope("Countries"), "australia", "brazil", "canada") + + output := task.ApplyTransform(s, fruits, countries) + + debug.Print(s, output) + + err := beamx.Run(ctx, p) + + if err != nil { + log.Exitf(ctx, "Failed to execute job: %v", err) + } +} diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.mod b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.mod new file mode 100644 index 000000000000..c0c99f3de73b --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.mod @@ -0,0 +1,26 @@ +// 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. + +module cogroupbykey + +go 1.14 + +require ( + github.com/apache/beam v2.20.0+incompatible + github.com/golang/protobuf v1.4.2 // indirect + github.com/google/go-cmp v0.4.1 // indirect + google.golang.org/api v0.25.0 // indirect + google.golang.org/grpc v1.29.1 // indirect +) diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.sum b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.sum new file mode 100644 index 000000000000..92a428884f45 --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/go.sum @@ -0,0 +1,314 @@ +cloud.google.com/go v0.26.0 h1:e0WKqKTd5BnrG8aKH3J3h+QvEIQtSUcf2n5UZ5ZgLtQ= +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.56.0 h1:WRz29PgAsVEyPSDHyk+0fpEkwEFyfhHn+JbksT6gIL4= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0 h1:xE3CPsOgttP4ACBePh79zTKALtXwn/Edhcr16R5hMWU= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0 h1:/May9ojXjRkPBNVrq+oWLqmWCkr4OU5uRY29bu0mRyQ= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0 h1:Lpy6hKgdcl7a3WGSfJIFmxmcdjSpP6OmBEfcOv1Y680= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0 h1:UDpwYIwla4jHGzZJaEJYx1tOejbgSoNqsAfHAUYe2r8= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/apache/beam v2.20.0+incompatible h1:YzP/+VyAnYdu4Wjh5EkBz3vUjVdE7vUPEZ6xijCJ2sk= +github.com/apache/beam v2.20.0+incompatible/go.mod h1:/8NX3Qi8vGstDLLaeaU7+lzVEu/ACaQhYjeefzQ0y1o= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1 h1:/exdXoGamhu5ONeUJH0deniYLWYvQwW66yvlfiiKTu0= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be h1:vEDujvNQGv4jgYKudGeI/+DAX4Jffq6hpD55MmoEvKs= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a h1:WXEvlFVvvGxCJLG6REjsT03iWnKLEWinaScsxF2Vm2o= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d h1:nc5K6ox/4lTFbMVSL9WRR81ixkcwXThoiF6yf+R9scA= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4 h1:kDtqNkeBrZb8B+atrj50B5XLHpzXXqcCdZPP/ApQ5NY= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.25.0 h1:LodzhlzZEUfhXzNUMIfVlf9Gr6Ua5MMtoFWh7+f47qA= +google.golang.org/api v0.25.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5 h1:tycE03LOZYQNhDpS27tcQdAzLCVMaj7QT2SXxebnpCM= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940 h1:MRHtG0U6SnaUb+s+LhNE1qt1FQ1wlhqr5E4usBKC0uA= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/pkg/task/task.go b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/pkg/task/task.go new file mode 100644 index 000000000000..c498b3585a1d --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/pkg/task/task.go @@ -0,0 +1,52 @@ +// 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 task + +import ( + "fmt" + "github.com/apache/beam/sdks/go/pkg/beam" +) + +func ApplyTransform(s beam.Scope, fruits beam.PCollection, countries beam.PCollection) beam.PCollection { + fruitsKV := beam.ParDo(s, func(word string) (string, string) { + return string(word[0]), word + }, fruits) + + countriesKV := beam.ParDo(s, func(word string) (string, string) { + return string(word[0]), word + }, countries) + + grouped := beam.CoGroupByKey(s, fruitsKV, countriesKV) + return beam.ParDo(s, func(key string, fruitsIter func(*string) bool, countriesIter func(*string) bool, emit func(string)) { + wa := &WordsAlphabet{ + Alphabet: key, + } + fruitsIter(&wa.Fruit) + countriesIter(&wa.Country) + emit(wa.String()) + }, grouped) +} + +type WordsAlphabet struct { + Alphabet string + Fruit string + Country string +} + +func (wa *WordsAlphabet) String() string { + return fmt.Sprintf("WordsAlphabet%+v", *wa) +} + diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-info.yaml b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-info.yaml new file mode 100644 index 000000000000..2aa604763256 --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-info.yaml @@ -0,0 +1,35 @@ +# +# 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. +# + +type: edu +files: +- name: test/task_test.go + visible: false +- name: cmd/main.go + visible: true +- name: go.mod + visible: false +- name: pkg/task/task.go + visible: true + placeholders: + - offset: 978 + length: 533 + placeholder_text: TODO() +- name: go.sum + visible: false diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-remote-info.yaml b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-remote-info.yaml new file mode 100644 index 000000000000..6846e3ac7a55 --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task-remote-info.yaml @@ -0,0 +1,2 @@ +id: 1342650 +update_date: Thu, 28 May 2020 22:33:24 UTC diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task.md b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task.md new file mode 100644 index 000000000000..9d2868abc44c --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/task.md @@ -0,0 +1,104 @@ + + +# CoGroupByKey + +CoGroupByKey performs a relational join of two or more key/value PCollections that have the same +key type. + +**Kata:** Implement a [beam.CoGroupByKey](https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#CoGroupByKey) +transform that join words by the first alphabetical letter, and then produces the string representation of the +WordsAlphabet model. + +

    + Refer to + beam.CoGroupByKey + to solve this problem. +
    + +
    + Refer to the Beam Programming Guide + + "CoGroupByKey" section for more information. +
    + +
    + Think of this problem in three stages. First, create key/value pairs of PCollections called KV + for fruits and countries, pairing the first character with the word. Next, apply CoGroupByKey to the KVs + followed by a ParDo. +
    + +
    + In the last lesson we learned how to make key/value PCollections called KV. Now we have + two to make from fruits and countries. + + To return as a KV, you can return two values from your DoFn. The first return value represents the Key, and + the second return value represents the Value. An example is shown below. + +``` +func doFn(element string) (string, string) { + key := string(element[0]) + value := element + return key, value +} +``` +
    + +
    + In the last lesson we learned that + + beam.GroupByKey takes a single KV. + beam.CoGroupByKey + takes more than one KV. +
    + +
    + Our final step in this problem requires a + beam.ParDo + with a DoFn that's different than what we've seen in previous lessons. In the previous step we should + have a PCollection acquired from CoGroupByKey. A ParDo for that PCollection expects a DoFn that looks + like the following. + + ``` + func doFn(key string, iterA func(*string) bool, iterB func(*string) bool, emit func(string)){ + ... + } + ``` + + Each `func(*string) bool` parameter above corresponds to each of the PCollection KVs that we provided to CoGroupByKey. + + `func(*string)` returns true if there is a value available when the pipeline invokes your doFn. + + Note that it takes a `*string` instead of a `string`. In Go, this means that to acquire its value, we need to + do this. + +``` + var v string + iterA(&v) + // do something with v +``` + + Not necessary for this task, though if you expected multiple values per key, you would do something like this. +``` + var v string + for iterA(&v) { + // do something with v + } +``` +
    diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/test/task_test.go b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/test/task_test.go new file mode 100644 index 000000000000..90a5912ebe22 --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/CoGroupByKey/test/task_test.go @@ -0,0 +1,55 @@ +// 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 task + +import ( + "cogroupbykey/pkg/task" + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/go/pkg/beam/testing/ptest" + "testing" +) + +func TestApplyTransform(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + type args struct { + fruits beam.PCollection + countries beam.PCollection + } + tests := []struct { + args args + want []interface{} + }{ + { + args: args{ + fruits: beam.Create(s.Scope("Fruits"), "apple", "banana", "cherry"), + countries: beam.Create(s.Scope("Countries"), "australia", "brazil", "canada"), + }, + want: []interface{}{ + "WordsAlphabet{Alphabet:a Fruit:apple Country:australia}", + "WordsAlphabet{Alphabet:b Fruit:banana Country:brazil}", + "WordsAlphabet{Alphabet:c Fruit:cherry Country:canada}", + }, + }, + } + for _, tt := range tests { + got := task.ApplyTransform(s, tt.args.fruits, tt.args.countries) + passert.Equals(s, got, tt.want...) + if err := ptest.Run(p); err != nil { + t.Error(err) + } + } +} diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/lesson-info.yaml b/learning/katas/go/Core Transforms/CoGroupByKey/lesson-info.yaml new file mode 100644 index 000000000000..273c0771cccf --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/lesson-info.yaml @@ -0,0 +1,21 @@ +# +# 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. +# + +content: +- CoGroupByKey diff --git a/learning/katas/go/Core Transforms/CoGroupByKey/lesson-remote-info.yaml b/learning/katas/go/Core Transforms/CoGroupByKey/lesson-remote-info.yaml new file mode 100644 index 000000000000..548d0f7247dc --- /dev/null +++ b/learning/katas/go/Core Transforms/CoGroupByKey/lesson-remote-info.yaml @@ -0,0 +1,3 @@ +id: 361640 +update_date: Thu, 28 May 2020 22:33:20 UTC +unit: 346169 diff --git a/learning/katas/go/Core Transforms/Flatten/Flatten/task-remote-info.yaml b/learning/katas/go/Core Transforms/Flatten/Flatten/task-remote-info.yaml new file mode 100644 index 000000000000..6d24d40ec8c0 --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/Flatten/task-remote-info.yaml @@ -0,0 +1,2 @@ +id: 1342651 +update_date: Thu, 28 May 2020 22:33:30 UTC diff --git a/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml b/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml new file mode 100644 index 000000000000..f7ed569d95fa --- /dev/null +++ b/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml @@ -0,0 +1,3 @@ +id: 361641 +update_date: Thu, 28 May 2020 22:33:26 UTC +unit: 346170 diff --git a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.mod b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.mod index 64758068e63d..89da9b4bcbf4 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.mod +++ b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.mod @@ -15,12 +15,11 @@ module pardo_onetomany -go 1.13 +go 1.14 require ( github.com/apache/beam v2.20.0+incompatible github.com/golang/protobuf v1.4.0 // indirect - github.com/googleapis/gax-go v2.0.2+incompatible // indirect go.opencensus.io v0.22.3 // indirect google.golang.org/api v0.22.0 // indirect google.golang.org/grpc v1.28.1 // indirect diff --git a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.sum b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.sum index 66ee82f3d108..709c1101db07 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.sum +++ b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/go.sum @@ -35,10 +35,9 @@ github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww= -github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= @@ -82,6 +81,7 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b h1:ag/x1USPSsqHud38I9BAC88qdNLDHHtQ4mlgQIZPPNA= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -102,6 +102,7 @@ google.golang.org/api v0.22.0 h1:J1Pl9P2lnmYFSJvgs70DKELqHNh8CNWXPbud4njEE2s= google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0 h1:KxkO13IPW4Lslp2bz+KHP2E3gtFlrIGNThxkZQ3g+4c= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml index 8e0e17b2c09f..6b6405a4ad16 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo OneToMany/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289189 -update_date: Wed, 20 May 2020 06:15:28 UTC +update_date: Thu, 28 May 2020 22:33:42 UTC diff --git a/learning/katas/go/Core Transforms/Map/ParDo Struct/go.mod b/learning/katas/go/Core Transforms/Map/ParDo Struct/go.mod index 0c0fddeae56e..7b5d3a2e31f7 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo Struct/go.mod +++ b/learning/katas/go/Core Transforms/Map/ParDo Struct/go.mod @@ -15,12 +15,11 @@ module mapelements -go 1.13 +go 1.14 require ( github.com/apache/beam v2.20.0+incompatible github.com/golang/protobuf v1.4.0 // indirect - github.com/googleapis/gax-go v2.0.2+incompatible // indirect go.opencensus.io v0.22.3 // indirect google.golang.org/api v0.22.0 // indirect google.golang.org/grpc v1.28.1 // indirect diff --git a/learning/katas/go/Core Transforms/Map/ParDo Struct/go.sum b/learning/katas/go/Core Transforms/Map/ParDo Struct/go.sum index 66ee82f3d108..709c1101db07 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo Struct/go.sum +++ b/learning/katas/go/Core Transforms/Map/ParDo Struct/go.sum @@ -35,10 +35,9 @@ github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww= -github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= @@ -82,6 +81,7 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b h1:ag/x1USPSsqHud38I9BAC88qdNLDHHtQ4mlgQIZPPNA= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -102,6 +102,7 @@ google.golang.org/api v0.22.0 h1:J1Pl9P2lnmYFSJvgs70DKELqHNh8CNWXPbud4njEE2s= google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0 h1:KxkO13IPW4Lslp2bz+KHP2E3gtFlrIGNThxkZQ3g+4c= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml index 5ca73c7400f6..368c38dad01a 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo Struct/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289190 -update_date: Wed, 20 May 2020 06:15:31 UTC +update_date: Thu, 28 May 2020 22:33:45 UTC diff --git a/learning/katas/go/Core Transforms/Map/ParDo/go.mod b/learning/katas/go/Core Transforms/Map/ParDo/go.mod index da50fd0795d4..4393de855643 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo/go.mod +++ b/learning/katas/go/Core Transforms/Map/ParDo/go.mod @@ -15,12 +15,11 @@ module pardo -go 1.13 +go 1.14 require ( github.com/apache/beam v2.20.0+incompatible github.com/golang/protobuf v1.4.0 // indirect - github.com/googleapis/gax-go v2.0.2+incompatible // indirect go.opencensus.io v0.22.3 // indirect google.golang.org/api v0.21.0 // indirect google.golang.org/grpc v1.28.1 // indirect diff --git a/learning/katas/go/Core Transforms/Map/ParDo/go.sum b/learning/katas/go/Core Transforms/Map/ParDo/go.sum index 22b190304fca..395e4a1ba737 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo/go.sum +++ b/learning/katas/go/Core Transforms/Map/ParDo/go.sum @@ -35,10 +35,9 @@ github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMyw github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww= -github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= @@ -82,6 +81,7 @@ golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b h1:ag/x1USPSsqHud38I9BAC88qdNLDHHtQ4mlgQIZPPNA= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -102,6 +102,7 @@ google.golang.org/api v0.21.0 h1:zS+Q/CJJnVlXpXQVIz+lH0ZT2lBuT2ac7XD8Y/3w6hY= google.golang.org/api v0.21.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0 h1:KxkO13IPW4Lslp2bz+KHP2E3gtFlrIGNThxkZQ3g+4c= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml b/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml index c5aaaae2a15f..42e50eda6d25 100644 --- a/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Map/ParDo/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1289188 -update_date: Wed, 20 May 2020 06:15:25 UTC +update_date: Thu, 28 May 2020 22:33:39 UTC diff --git a/learning/katas/go/Core Transforms/section-info.yaml b/learning/katas/go/Core Transforms/section-info.yaml index 3fe207041313..d228f9a30d38 100644 --- a/learning/katas/go/Core Transforms/section-info.yaml +++ b/learning/katas/go/Core Transforms/section-info.yaml @@ -20,4 +20,5 @@ content: - Map - GroupByKey -- Flatten \ No newline at end of file +- CoGroupByKey +- Flatten diff --git a/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/go.mod b/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/go.mod index 6cf809a0c9b5..696bd8f55c03 100644 --- a/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/go.mod +++ b/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/go.mod @@ -15,7 +15,7 @@ module hello_beam_test -go 1.13 +go 1.14 require ( github.com/apache/beam v2.19.0+incompatible diff --git a/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/task-remote-info.yaml b/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/task-remote-info.yaml index a75da50ff181..eaf8e5387bd1 100644 --- a/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/task-remote-info.yaml +++ b/learning/katas/go/Introduction/Hello Beam/Hello Beam Test/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1236832 -update_date: Tue, 05 May 2020 02:02:03 UTC +update_date: Thu, 28 May 2020 22:33:36 UTC diff --git a/learning/katas/go/Introduction/Hello Beam/Hello Beam/go.mod b/learning/katas/go/Introduction/Hello Beam/Hello Beam/go.mod index 13bb5997a52f..233fe5c13c7e 100644 --- a/learning/katas/go/Introduction/Hello Beam/Hello Beam/go.mod +++ b/learning/katas/go/Introduction/Hello Beam/Hello Beam/go.mod @@ -15,7 +15,7 @@ module hello_beam -go 1.13 +go 1.14 require ( github.com/apache/beam v2.19.0+incompatible diff --git a/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml b/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml index 46551fb4c9fd..7f138de415ba 100644 --- a/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml +++ b/learning/katas/go/Introduction/Hello Beam/Hello Beam/task-remote-info.yaml @@ -1,2 +1,2 @@ id: 1236831 -update_date: Wed, 20 May 2020 06:15:22 UTC +update_date: Thu, 28 May 2020 22:33:33 UTC diff --git a/learning/katas/go/course-remote-info.yaml b/learning/katas/go/course-remote-info.yaml index 2651d8336da7..b875f83687ed 100644 --- a/learning/katas/go/course-remote-info.yaml +++ b/learning/katas/go/course-remote-info.yaml @@ -1,2 +1,2 @@ id: 70387 -update_date: Tue, 05 May 2020 02:02:55 UTC +update_date: Wed, 20 May 2020 06:16:18 UTC From 1bcb551a7b212989b08a518b5c8ebdf52d11775e Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 29 May 2020 10:54:53 -0700 Subject: [PATCH 120/151] [BEAM-10110] Propagate ids for custom coders. (#11832) * [BEAM-10110] Propagate ids for custom coders. * [BEAM-10110] In coder strings, use ids if present. --- sdks/go/pkg/beam/core/graph/coder/coder.go | 17 +++++++++++++++-- sdks/go/pkg/beam/core/runtime/graphx/coder.go | 3 +++ .../go/pkg/beam/core/runtime/graphx/dataflow.go | 16 ++++++++++------ 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/sdks/go/pkg/beam/core/graph/coder/coder.go b/sdks/go/pkg/beam/core/graph/coder/coder.go index 61ebdc6f3d37..2ba07d72c882 100644 --- a/sdks/go/pkg/beam/core/graph/coder/coder.go +++ b/sdks/go/pkg/beam/core/graph/coder/coder.go @@ -45,6 +45,8 @@ type CustomCoder struct { // Dec is the decoding function: []byte -> T. It may optionally take a // reflect.Type parameter and return an error as well. Dec *funcx.Fn + + ID string // (optional) This coder's ID if translated from a pipeline proto. } // TODO(herohde) 5/16/2017: do we want/need to allow user coders that follow the @@ -76,7 +78,10 @@ func (c *CustomCoder) Equals(o *CustomCoder) bool { } func (c *CustomCoder) String() string { - return fmt.Sprintf("%v[%v]", c.Type, c.Name) + if c.ID == "" { + return fmt.Sprintf("%v[%v]", c.Type, c.Name) + } + return fmt.Sprintf("%v[%v;%v]", c.Type, c.Name, c.ID) } // Type signatures of encode/decode for verification. @@ -187,6 +192,8 @@ type Coder struct { Components []*Coder // WindowedValue, KV, CoGBK Custom *CustomCoder // Custom Window *WindowCoder // WindowedValue + + ID string // (optional) This coder's ID if translated from a pipeline proto. } // Equals returns true iff the two coders are equal. It assumes that @@ -224,10 +231,16 @@ func (c *Coder) String() string { return "$" } if c.Custom != nil { - return c.Custom.String() + if c.ID == "" { + return c.Custom.String() + } + return fmt.Sprintf("%v;%v", c.Custom, c.ID) } ret := fmt.Sprintf("%v", c.Kind) + if c.ID != "" { + ret = fmt.Sprintf("%v;%v", c.Kind, c.ID) + } if len(c.Components) > 0 { var args []string for _, elm := range c.Components { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder.go b/sdks/go/pkg/beam/core/runtime/graphx/coder.go index b00581f0c367..212ad3b78d4c 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder.go @@ -106,6 +106,7 @@ func NewCoderUnmarshaller(m map[string]*pb.Coder) *CoderUnmarshaller { } } +// Coders unmarshals a list of coder ids. func (b *CoderUnmarshaller) Coders(ids []string) ([]*coder.Coder, error) { coders := make([]*coder.Coder, len(ids)) for i, id := range ids { @@ -133,6 +134,7 @@ func (b *CoderUnmarshaller) Coder(id string) (*coder.Coder, error) { if err != nil { return nil, errors.WithContextf(err, "unmarshalling coder %v", id) } + ret.ID = id b.coders[id] = ret return ret, nil @@ -256,6 +258,7 @@ func (b *CoderUnmarshaller) makeCoder(c *pb.Coder) (*coder.Coder, error) { if err != nil { return nil, err } + custom.ID = components[0] t := typex.New(custom.Type) return &coder.Coder{Kind: coder.Custom, T: t, Custom: custom}, nil diff --git a/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go b/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go index da6d9b520e77..364a849962cb 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/dataflow.go @@ -29,11 +29,12 @@ import ( // CoderRef defines the (structured) Coder in serializable form. It is // an artifact of the CloudObject encoding. type CoderRef struct { - Type string `json:"@type,omitempty"` - Components []*CoderRef `json:"component_encodings,omitempty"` - IsWrapper bool `json:"is_wrapper,omitempty"` - IsPairLike bool `json:"is_pair_like,omitempty"` - IsStreamLike bool `json:"is_stream_like,omitempty"` + Type string `json:"@type,omitempty"` + Components []*CoderRef `json:"component_encodings,omitempty"` + IsWrapper bool `json:"is_wrapper,omitempty"` + IsPairLike bool `json:"is_pair_like,omitempty"` + IsStreamLike bool `json:"is_stream_like,omitempty"` + PipelineProtoCoderID string `json:"pipeline_proto_coder_id,omitempty"` } // Exported types are used for translation lookup. @@ -92,7 +93,10 @@ func EncodeCoderRef(c *coder.Coder) (*CoderRef, error) { if err != nil { return nil, err } - return &CoderRef{Type: lengthPrefixType, Components: []*CoderRef{{Type: data}}}, nil + return &CoderRef{ + Type: lengthPrefixType, + Components: []*CoderRef{{Type: data, PipelineProtoCoderID: c.Custom.ID}}, + }, nil case coder.KV: if len(c.Components) != 2 { From 05df70322217db17c03e00eca97b330fb9b1eef1 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Fri, 29 May 2020 14:19:00 -0400 Subject: [PATCH 121/151] [BEAM-10154] Remove stray version number. --- website/www/site/content/en/documentation/dsls/sql/overview.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/dsls/sql/overview.md b/website/www/site/content/en/documentation/dsls/sql/overview.md index fc89197ef884..f91037b3e9da 100644 --- a/website/www/site/content/en/documentation/dsls/sql/overview.md +++ b/website/www/site/content/en/documentation/dsls/sql/overview.md @@ -38,7 +38,7 @@ There are two additional concepts you need to know to use SQL in your pipeline: - [SqlTransform](https://beam.apache.org/releases/javadoc/{{< param release_latest >}}/index.html?org/apache/beam/sdk/extensions/sql/SqlTransform.html): the interface for creating `PTransforms` from SQL queries. - [Row](https://beam.apache.org/releases/javadoc/{{< param release_latest >}}/index.html?org/apache/beam/sdk/values/Row.html): the type of elements that Beam SQL operates on. A `PCollection` plays the role of a table. -{{< param release_latest >}} + ## Walkthrough The [SQL pipeline walkthrough](/documentation/dsls/sql/walkthrough) works through how to use Beam SQL with example code. From 21848c63978892ea2b021529da270ababc6fffb2 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 1 May 2020 14:04:31 -0700 Subject: [PATCH 122/151] Moves apply_WriteToBigQuery to override This will throw an exception in the case that there is a consumer for the WriteToBigQuery that will be replaced. This is because the replacement write outputs a PDone instead of a dict. Change-Id: If782bf1276da3ae739b7d44dad9bea4c12e875f5 --- .../runners/dataflow/dataflow_runner.py | 40 +----- .../runners/dataflow/dataflow_runner_test.py | 61 ++++++++ .../runners/dataflow/ptransform_overrides.py | 136 ++++++++++++++++++ 3 files changed, 200 insertions(+), 37 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 4cddd3318a82..063dd6ec893a 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -26,7 +26,6 @@ from __future__ import division import base64 -import json import logging import os import subprocess @@ -464,6 +463,9 @@ def run_pipeline(self, pipeline, options): # any added PTransforms. pipeline.replace_all(DataflowRunner._PTRANSFORM_OVERRIDES) + from apache_beam.runners.dataflow.ptransform_overrides import WriteToBigQueryPTransformOverride + pipeline.replace_all([WriteToBigQueryPTransformOverride(pipeline, options)]) + if (apiclient._use_fnapi(options) and not apiclient._use_unified_worker(options)): pipeline.replace_all(DataflowRunner._JRH_PTRANSFORM_OVERRIDES) @@ -805,42 +807,6 @@ def run_Flatten(self, transform_node, options): PropertyNames.OUTPUT_NAME: PropertyNames.OUT }]) - def apply_WriteToBigQuery(self, transform, pcoll, options): - # Make sure this is the WriteToBigQuery class that we expected, and that - # users did not specifically request the new BQ sink by passing experiment - # flag. - - # TODO(BEAM-6928): Remove this function for release 2.14.0. - experiments = options.view_as(DebugOptions).experiments or [] - from apache_beam.runners.dataflow.internal import apiclient - use_fnapi = apiclient._use_fnapi(options) - if (not isinstance(transform, beam.io.WriteToBigQuery) or use_fnapi or - 'use_beam_bq_sink' in experiments): - return self.apply_PTransform(transform, pcoll, options) - if transform.schema == beam.io.gcp.bigquery.SCHEMA_AUTODETECT: - raise RuntimeError( - 'Schema auto-detection is not supported on the native sink') - standard_options = options.view_as(StandardOptions) - if standard_options.streaming: - if (transform.write_disposition == - beam.io.BigQueryDisposition.WRITE_TRUNCATE): - raise RuntimeError('Can not use write truncation mode in streaming') - return self.apply_PTransform(transform, pcoll, options) - else: - from apache_beam.io.gcp.bigquery_tools import parse_table_schema_from_json - schema = None - if transform.schema: - schema = parse_table_schema_from_json(json.dumps(transform.schema)) - return pcoll | 'WriteToBigQuery' >> beam.io.Write( - beam.io.BigQuerySink( - transform.table_reference.tableId, - transform.table_reference.datasetId, - transform.table_reference.projectId, - schema, - transform.create_disposition, - transform.write_disposition, - kms_key=transform.kms_key)) - # TODO(srohde): Remove this after internal usages have been removed. def apply_GroupByKey(self, transform, pcoll, options): return transform.expand(pcoll) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 16ea3362661a..4d0f116fbc48 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -752,6 +752,67 @@ def test_gbk_translation(self): self.assertEqual( gbk_step[u'properties']['output_info'], expected_output_info) + def test_write_bigquery_translation(self): + runner = DataflowRunner() + + with beam.Pipeline(runner=runner, + options=PipelineOptions(self.default_properties)) as p: + # pylint: disable=expression-not-assigned + p | beam.Create([1]) | beam.io.WriteToBigQuery('some.table') + + job_dict = json.loads(str(runner.job)) + + expected_step = { + "kind": "ParallelWrite", + "name": "s2", + "properties": { + "create_disposition": "CREATE_IF_NEEDED", + "dataset": "some", + "display_data": [], + "encoding": { + "@type": "kind:windowed_value", + "component_encodings": [{ + "component_encodings": [], + "pipeline_proto_coder_id": "ref_Coder_RowAsDictJsonCoder_4" + }, { + "@type": "kind:global_window" + }], + "is_wrapper": True + }, + "format": "bigquery", + "parallel_input": { + "@type": "OutputReference", + "output_name": "out", + "step_name": "s1" + }, + "table": "table", + "user_name": "WriteToBigQuery/Write/NativeWrite", + "write_disposition": "WRITE_APPEND" + } + } + job_dict = json.loads(str(runner.job)) + write_step = [ + s for s in job_dict[u'steps'] + if s[u'properties'][u'user_name'].startswith('WriteToBigQuery') + ][0] + + # Delete the @type field because in this case it is a hash which may change + # depending on the pickling version. + step_encoding = write_step[u'properties'][u'encoding'] + del step_encoding[u'component_encodings'][0][u'@type'] + self.assertEqual(expected_step, write_step) + + def test_write_bigquery_failed_translation(self): + """Tests that WriteToBigQuery cannot have any consumers if replaced.""" + runner = DataflowRunner() + + with self.assertRaises(ValueError): + with beam.Pipeline(runner=runner, + options=PipelineOptions(self.default_properties)) as p: + # pylint: disable=expression-not-assigned + out = p | beam.Create([1]) | beam.io.WriteToBigQuery('some.table') + out['FailedRows'] | 'MyTransform' >> beam.Map(lambda _: _) + class CustomMergingWindowFn(window.WindowFn): def assign(self, assign_context): diff --git a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py index 9d6e5d35467f..75b1db0b6341 100644 --- a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py +++ b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py @@ -21,6 +21,8 @@ from __future__ import absolute_import +from apache_beam.options.pipeline_options import DebugOptions +from apache_beam.options.pipeline_options import StandardOptions from apache_beam.pipeline import PTransformOverride @@ -184,3 +186,137 @@ def expand(self, pbegin): # will choose the incorrect coder for this transform. return Read(ptransform.source).with_output_types( ptransform.source.coder.to_type_hint()) + + +class WriteToBigQueryPTransformOverride(PTransformOverride): + def __init__(self, pipeline, options): + super(WriteToBigQueryPTransformOverride, self).__init__() + self.options = options + self.outputs = [] + + self._check_bq_outputs(pipeline) + + def _check_bq_outputs(self, pipeline): + """Checks that there are no consumers if the transform will be replaced. + + The WriteToBigQuery replacement is the native BigQuerySink which has an + output of a PDone. The original transform, however, returns a dict. The user + may be inadvertantly using the dict output which will have no side-effects + or fail pipeline construction esoterically. This checks the outputs and + gives a user-friendsly error. + """ + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.pipeline import PipelineVisitor + from apache_beam.io import WriteToBigQuery + + # First, retrieve all the outpts from all the WriteToBigQuery transforms + # that will be replaced. Later, we will use these to make sure no one + # consumes these. + class GetWriteToBqOutputsVisitor(PipelineVisitor): + def __init__(self, matches): + self.matches = matches + self.outputs = set() + + def enter_composite_transform(self, transform_node): + # Only add outputs that are going to be replaced. + if self.matches(transform_node): + self.outputs.update(set(transform_node.outputs.values())) + + outputs_visitor = GetWriteToBqOutputsVisitor(self.matches) + pipeline.visit(outputs_visitor) + + # Finally, verify that there are no consumers to the previously found + # outputs. + class VerifyWriteToBqOutputsVisitor(PipelineVisitor): + def __init__(self, outputs): + self.outputs = outputs + + def enter_composite_transform(self, transform_node): + self.visit_transform(transform_node) + + def visit_transform(self, transform_node): + if [o for o in self.outputs if o in transform_node.inputs]: + raise ValueError( + 'WriteToBigQuery was being replaced with the native ' + 'BigQuerySink, but the transform "{}" has an input which will be ' + 'replaced with a PDone. To fix, please remove all transforms ' + 'that read from any WriteToBigQuery transforms.'.format( + transform_node.full_label)) + + pipeline.visit(VerifyWriteToBqOutputsVisitor(outputs_visitor.outputs)) + + def matches(self, applied_ptransform): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam import io + from apache_beam.runners.dataflow.internal import apiclient + + transform = applied_ptransform.transform + if (not isinstance(transform, io.WriteToBigQuery) or + getattr(transform, 'override', False)): + return False + + use_fnapi = apiclient._use_fnapi(self.options) + experiments = self.options.view_as(DebugOptions).experiments or [] + if (use_fnapi or 'use_beam_bq_sink' in experiments): + return False + + if transform.schema == io.gcp.bigquery.SCHEMA_AUTODETECT: + raise RuntimeError( + 'Schema auto-detection is not supported on the native sink') + + # The replacement is only valid for Batch. + standard_options = self.options.view_as(StandardOptions) + if standard_options.streaming: + if transform.write_disposition == io.BigQueryDisposition.WRITE_TRUNCATE: + raise RuntimeError('Can not use write truncation mode in streaming') + return False + + self.outputs = list(applied_ptransform.outputs.keys()) + return True + + def get_replacement_transform(self, ptransform): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam import io + + class WriteToBigQuery(io.WriteToBigQuery): + override = True + + def __init__(self, transform, outputs): + self.transform = transform + self.outputs = outputs + + def __getattr__(self, name): + """Returns the given attribute from the parent. + + This allows this transform to act like a WriteToBigQuery transform + without having to construct a new WriteToBigQuery transform. + """ + return self.transform.__getattribute__(name) + + def expand(self, pcoll): + from apache_beam.io.gcp.bigquery_tools import parse_table_schema_from_json + import json + + schema = None + if self.schema: + schema = parse_table_schema_from_json(json.dumps(self.schema)) + + out = pcoll | io.Write( + io.BigQuerySink( + self.table_reference.tableId, + self.table_reference.datasetId, + self.table_reference.projectId, + schema, + self.create_disposition, + self.write_disposition, + kms_key=self.kms_key)) + + # The WriteToBigQuery can have different outputs depending on if it's + # Batch or Streaming. This retrieved the output keys from the node and + # is replacing them here to be consistent. + return {key: out for key in self.outputs} + + return WriteToBigQuery(ptransform, self.outputs) From 12878fce8a096fd4d4f82d8cc6a2f9343f92b35b Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Fri, 29 May 2020 11:56:38 -0700 Subject: [PATCH 123/151] [BEAM-10121] Python RowCoder doesn't support nested structs (#11841) * Rename TEST_CASE * Add support for nested structs --- sdks/python/apache_beam/coders/row_coder.py | 2 ++ .../apache_beam/coders/row_coder_test.py | 22 +++++++++++++------ 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py index 03114976c78e..8c41891d0d8a 100644 --- a/sdks/python/apache_beam/coders/row_coder.py +++ b/sdks/python/apache_beam/coders/row_coder.py @@ -103,6 +103,8 @@ def coder_from_type(field_type): elif type_info == "array_type": return IterableCoder( RowCoder.coder_from_type(field_type.array_type.element_type)) + elif type_info == "row_type": + return RowCoder(field_type.row_type.schema) # The Java SDK supports several more types, but the coders are not yet # standard, and are not implemented in Python. diff --git a/sdks/python/apache_beam/coders/row_coder_test.py b/sdks/python/apache_beam/coders/row_coder_test.py index 0ffd98338190..8eb7ee26dcfb 100644 --- a/sdks/python/apache_beam/coders/row_coder_test.py +++ b/sdks/python/apache_beam/coders/row_coder_test.py @@ -49,9 +49,9 @@ class RowCoderTest(unittest.TestCase): - TEST_CASE = Person("Jon Snow", 23, None, ["crow", "wildling"]) - TEST_CASES = [ - TEST_CASE, + JON_SNOW = Person("Jon Snow", 23, None, ["crow", "wildling"]) + PEOPLE = [ + JON_SNOW, Person("Daenerys Targaryen", 25, "Westeros", ["Mother of Dragons"]), Person("Michael Bluth", 30, None, []) ] @@ -60,7 +60,7 @@ def test_create_row_coder_from_named_tuple(self): expected_coder = RowCoder(typing_to_runner_api(Person).row_type.schema) real_coder = coders_registry.get_coder(Person) - for test_case in self.TEST_CASES: + for test_case in self.PEOPLE: self.assertEqual( expected_coder.encode(test_case), real_coder.encode(test_case)) @@ -90,7 +90,7 @@ def test_create_row_coder_from_schema(self): ]) coder = RowCoder(schema) - for test_case in self.TEST_CASES: + for test_case in self.PEOPLE: self.assertEqual(test_case, coder.decode(coder.encode(test_case))) @unittest.skip( @@ -182,9 +182,17 @@ def test_row_coder_in_pipeine(self): with TestPipeline() as p: res = ( p - | beam.Create(self.TEST_CASES) + | beam.Create(self.PEOPLE) | beam.Filter(lambda person: person.name == "Jon Snow")) - assert_that(res, equal_to([self.TEST_CASE])) + assert_that(res, equal_to([self.JON_SNOW])) + + def test_row_coder_nested_struct(self): + Pair = typing.NamedTuple('Pair', [('left', Person), ('right', Person)]) + + value = Pair(self.PEOPLE[0], self.PEOPLE[1]) + coder = RowCoder(typing_to_runner_api(Pair).row_type.schema) + + self.assertEqual(value, coder.decode(coder.encode(value))) if __name__ == "__main__": From b365680881dd82f1ea905eb24ab428372b8b930b Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 29 May 2020 12:02:58 -0700 Subject: [PATCH 124/151] Skip test if py=3.8 see BEAM-9754 Change-Id: Ifb2480e9d208d2bcf643c0edaf730fc2479a50fb --- .../apache_beam/runners/dataflow/dataflow_runner_test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 4d0f116fbc48..36e222d90b19 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -752,6 +752,9 @@ def test_gbk_translation(self): self.assertEqual( gbk_step[u'properties']['output_info'], expected_output_info) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_write_bigquery_translation(self): runner = DataflowRunner() @@ -802,6 +805,9 @@ def test_write_bigquery_translation(self): del step_encoding[u'component_encodings'][0][u'@type'] self.assertEqual(expected_step, write_step) + @unittest.skipIf( + sys.version_info.minor == 8, + 'Doesn\'t work on Python 3.8, see: BEAM-9754') def test_write_bigquery_failed_translation(self): """Tests that WriteToBigQuery cannot have any consumers if replaced.""" runner = DataflowRunner() From d87fc525e820f78c4546f3f0f5bd5218e3ce323f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 29 May 2020 12:24:48 -0700 Subject: [PATCH 125/151] [BEAM-7746] Fix type errors and enable checks for apache_beam.dataframe.* (#11632) --- sdks/python/apache_beam/dataframe/convert.py | 41 +++++++++++++------ sdks/python/apache_beam/dataframe/doctests.py | 5 ++- .../apache_beam/dataframe/frame_base.py | 3 +- .../apache_beam/dataframe/transforms.py | 26 ++++++++++-- sdks/python/mypy.ini | 3 -- 5 files changed, 57 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/convert.py b/sdks/python/apache_beam/dataframe/convert.py index c55fa9efa20c..f0c5824b27f1 100644 --- a/sdks/python/apache_beam/dataframe/convert.py +++ b/sdks/python/apache_beam/dataframe/convert.py @@ -17,12 +17,21 @@ from __future__ import absolute_import import inspect +from typing import TYPE_CHECKING +from typing import Any +from typing import Dict +from typing import Tuple +from typing import Union from apache_beam import pvalue from apache_beam.dataframe import expressions from apache_beam.dataframe import frame_base from apache_beam.dataframe import transforms +if TYPE_CHECKING: + # pylint: disable=ungrouped-imports + import pandas + # TODO: Or should this be called as_dataframe? def to_dataframe( @@ -49,9 +58,9 @@ def to_dataframe( # TODO: Or should this be called from_dataframe? def to_pcollection( - *dataframes, # type: Tuple[frame_base.DeferredFrame] + *dataframes, # type: frame_base.DeferredFrame **kwargs): - # type: (...) -> Union[pvalue.PCollection, Tuple[pvalue.PCollection]] + # type: (...) -> Union[pvalue.PCollection, Tuple[pvalue.PCollection, ...]] """Converts one or more deferred dataframe-like objects back to a PCollection. @@ -67,18 +76,23 @@ def to_pcollection( if label is None: # Attempt to come up with a reasonable, stable label by retrieving the name # of these variables in the calling context. - previous_frame = inspect.currentframe().f_back + current_frame = inspect.currentframe() + if current_frame is None: + label = 'ToDataframe(...)' + + else: + previous_frame = current_frame.f_back - def name(obj): - for key, value in previous_frame.f_locals.items(): - if obj is value: - return key - for key, value in previous_frame.f_globals.items(): - if obj is value: - return key - return '...' + def name(obj): + for key, value in previous_frame.f_locals.items(): + if obj is value: + return key + for key, value in previous_frame.f_globals.items(): + if obj is value: + return key + return '...' - label = 'ToDataframe(%s)' % ', '.join(name(e) for e in dataframes) + label = 'ToDataframe(%s)' % ', '.join(name(e) for e in dataframes) def extract_input(placeholder): if not isinstance(placeholder._reference, pvalue.PCollection): @@ -91,7 +105,8 @@ def extract_input(placeholder): results = {p: extract_input(p) for p in placeholders } | label >> transforms._DataframeExpressionsTransform( - dict((ix, df._expr) for ix, df in enumerate(dataframes))) + dict((ix, df._expr) for ix, df in enumerate( + dataframes))) # type: Dict[Any, pvalue.PCollection] if len(results) == 1 and not always_return_tuple: return results[0] else: diff --git a/sdks/python/apache_beam/dataframe/doctests.py b/sdks/python/apache_beam/dataframe/doctests.py index a81494438037..030a58d101e7 100644 --- a/sdks/python/apache_beam/dataframe/doctests.py +++ b/sdks/python/apache_beam/dataframe/doctests.py @@ -43,6 +43,9 @@ import contextlib import doctest import re +from typing import Any +from typing import Dict +from typing import List import numpy as np import pandas as pd @@ -136,7 +139,7 @@ class _InMemoryResultRecorder(object): """ # Class-level value to survive pickling. - _ALL_RESULTS = {} + _ALL_RESULTS = {} # type: Dict[str, List[Any]] def __init__(self): self._id = id(self) diff --git a/sdks/python/apache_beam/dataframe/frame_base.py b/sdks/python/apache_beam/dataframe/frame_base.py index 275d2b99e275..61b399a8811a 100644 --- a/sdks/python/apache_beam/dataframe/frame_base.py +++ b/sdks/python/apache_beam/dataframe/frame_base.py @@ -17,6 +17,7 @@ from __future__ import absolute_import import inspect +from typing import Dict import pandas as pd @@ -25,7 +26,7 @@ class DeferredFrame(object): - _pandas_type_map = {} + _pandas_type_map = {} # type: Dict[type, type] def __init__(self, expr): self._expr = expr diff --git a/sdks/python/apache_beam/dataframe/transforms.py b/sdks/python/apache_beam/dataframe/transforms.py index 31b62f1405fb..8c8943d73245 100644 --- a/sdks/python/apache_beam/dataframe/transforms.py +++ b/sdks/python/apache_beam/dataframe/transforms.py @@ -16,6 +16,15 @@ from __future__ import absolute_import +from typing import TYPE_CHECKING +from typing import Any +from typing import Dict +from typing import List +from typing import Mapping +from typing import Tuple +from typing import TypeVar +from typing import Union + import pandas as pd import apache_beam as beam @@ -23,6 +32,12 @@ from apache_beam.dataframe import expressions from apache_beam.dataframe import frames # pylint: disable=unused-import +if TYPE_CHECKING: + # pylint: disable=ungrouped-imports + from apache_beam.pvalue import PCollection + +T = TypeVar('T') + class DataframeTransform(transforms.PTransform): """A PTransform for applying function that takes and returns dataframes @@ -82,8 +97,8 @@ def expand(self, inputs): def _apply_deferred_ops( self, - inputs, # type: Dict[PlaceholderExpr, PCollection] - outputs, # type: Dict[Any, Expression] + inputs, # type: Dict[expressions.Expression, PCollection] + outputs, # type: Dict[Any, expressions.Expression] ): # -> Dict[Any, PCollection] """Construct a Beam graph that evaluates a set of expressions on a set of input PCollections. @@ -248,7 +263,12 @@ def _dict_union(dicts): return result -def _flatten(valueish, root=()): +def _flatten( + valueish, # type: Union[T, List[T], Tuple[T], Dict[Any, T]] + root=(), # type: Tuple[Any, ...] + ): + # type: (...) -> Mapping[Tuple[Any, ...], T] + """Given a nested structure of dicts, tuples, and lists, return a flat dictionary where the values are the leafs and the keys are the "paths" to these leaves. diff --git a/sdks/python/mypy.ini b/sdks/python/mypy.ini index 151eebf6b44b..1b24e6cd4ecd 100644 --- a/sdks/python/mypy.ini +++ b/sdks/python/mypy.ini @@ -67,9 +67,6 @@ ignore_errors = true [mypy-apache_beam.coders.*] ignore_errors = true -[mypy-apache_beam.dataframe.*] -ignore_errors = true - [mypy-apache_beam.io.*] ignore_errors = true From a0297da609956b429eaf5f34bea03df89b067022 Mon Sep 17 00:00:00 2001 From: Rajat Mittal Date: Sat, 30 May 2020 11:42:13 -0500 Subject: [PATCH 126/151] Update programming-guide.md --- website/www/site/content/en/documentation/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 82bd0534633e..067a3a33f826 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -2020,7 +2020,7 @@ Most structured records share some common characteristics: languages: int, long, string, etc. * Often a field type can be marked as optional (sometimes referred to as nullable) or required. -Oten records have a nested structure. A nested structure occurs when a field itself has subfields so the +Often records have a nested structure. A nested structure occurs when a field itself has subfields so the type of the field itself has a schema. Fields that are array or map types is also a common feature of these structured records. From 6598f10df293951f264ef9b94a7cf19ca47b180a Mon Sep 17 00:00:00 2001 From: Rajat Mittal Date: Sat, 30 May 2020 12:34:31 -0500 Subject: [PATCH 127/151] Update programming-guide.md --- .../site/content/en/documentation/programming-guide.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 067a3a33f826..a9f5f0962341 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -2655,7 +2655,7 @@ field as a top-level field. Both top-level and nested fields can be selected. Fo could select only the userId and streetAddress fields as follows {{< highlight java >}} -purchases.apply(Select.fieldNames("userId", shippingAddress.streetAddress")); +purchases.apply(Select.fieldNames("userId", "shippingAddress.streetAddress")); {{< /highlight >}} The resulting `PCollection` will have the following schema @@ -2683,7 +2683,7 @@ The resulting `PCollection` will have the following schema The same is true for wildcard selections. The following {{< highlight java >}} -purchases.apply(Select.fieldNames("userId", shippingAddress.*")); +purchases.apply(Select.fieldNames("userId", "shippingAddress.*")); {{< /highlight >}} Will result in the following schema @@ -2729,7 +2729,7 @@ top-level field in the resulting row. This means that if multiple fields are sel selected field will appear as its own array field. For example {{< highlight java >}} -purchases.apply(Select.fieldNames( "transactions.bank", transactions.purchaseAmount")); +purchases.apply(Select.fieldNames( "transactions.bank", "transactions.purchaseAmount")); {{< /highlight >}} Will result in the following schema @@ -2832,7 +2832,7 @@ The simplest usage of `Group` specifies no aggregations, in which case all input are grouped together into an `ITERABLE` field. For example {{< highlight java >}} -purchases.apply(Group.byFieldNames("userId", shippingAddress.streetAddress")); +purchases.apply(Group.byFieldNames("userId", "shippingAddress.streetAddress")); {{< /highlight >}} The output schema of this is: @@ -2863,7 +2863,7 @@ The names of the key and values fields in the output schema can be controlled us builders, as follows: {{< highlight java >}} -purchases.apply(Group.byFieldNames("userId", shippingAddress.streetAddress") +purchases.apply(Group.byFieldNames("userId", "shippingAddress.streetAddress") .withKeyField("userAndStreet") .withValueField("matchingPurchases")); {{< /highlight >}} From 36892965effd2f7298960a7beab74a78cc6c742b Mon Sep 17 00:00:00 2001 From: David Moravek Date: Mon, 1 Jun 2020 08:30:38 +0200 Subject: [PATCH 128/151] [BEAM-8608] Code review. Fix checkstyle. --- .../java/org/apache/beam/runners/flink/FlinkCapabilities.java | 4 ++-- .../java/org/apache/beam/runners/flink/FlinkCapabilities.java | 4 ++-- .../beam/runners/flink/FlinkBatchTransformTranslators.java | 4 ++-- .../java/build-tools/src/main/resources/beam/suppressions.xml | 1 + 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java b/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java index aabdc977e995..1b56c72946a9 100644 --- a/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java +++ b/runners/flink/1.10/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java @@ -23,12 +23,12 @@ public class FlinkCapabilities { /** * Support for outputting elements in close method of chained drivers. * - *

    {@see } for more + *

    {@see FLINK-14709} for more * details. * * @return True if feature is supported. */ - public static boolean supportsOutputInTearDown() { + public static boolean supportsOutputDuringClosing() { return true; } } diff --git a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java index 02586e38a262..e1d2a44f9597 100644 --- a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java +++ b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/FlinkCapabilities.java @@ -23,12 +23,12 @@ public class FlinkCapabilities { /** * Support for outputting elements in close method of chained drivers. * - *

    {@see } for more + *

    {@see FLINK-14709} for more * details. * * @return True if feature is supported. */ - public static boolean supportsOutputInTearDown() { + public static boolean supportsOutputDuringClosing() { return false; } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 9ddf5049dc01..8e3b623850be 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -653,11 +653,11 @@ public void translateNode( doFnSchemaInformation, sideInputMapping); - if (FlinkCapabilities.supportsOutputInTearDown()) { + if (FlinkCapabilities.supportsOutputDuringClosing()) { outputDataSet = new FlatMapOperator<>(inputDataSet, typeInformation, doFnWrapper, fullName); } else { - // This can be removed once we drop support for 1.9 and 1.10 versions. + // This can be removed once we drop support for 1.8 and 1.9 versions. outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, fullName); } diff --git a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml index fea30fb08615..751466f5a446 100644 --- a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml @@ -95,6 +95,7 @@ + From 1b1408540bb050d17491c71d77f98c403ae70236 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Mon, 1 Jun 2020 10:04:58 +0200 Subject: [PATCH 129/151] [BEAM-9421] Better documentation of output results from AnnotateText transform (#11776) --- .../apache_beam/examples/snippets/snippets.py | 91 ++++++++++ .../en/documentation/patterns/ai-platform.md | 159 ++++++++++++++++++ .../en/documentation/patterns/overview.md | 3 + .../section-menu/en/documentation.html | 27 +-- 4 files changed, 267 insertions(+), 13 deletions(-) create mode 100644 website/www/site/content/en/documentation/patterns/ai-platform.md diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index c2ca40117021..29d7577b9270 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -37,6 +37,7 @@ import argparse import base64 +import json from builtins import object from builtins import range from decimal import Decimal @@ -51,6 +52,13 @@ from apache_beam.testing.util import equal_to from apache_beam.transforms.core import PTransform +# Protect against environments where Google Cloud Natural Language client is +# not available. +try: + from apache_beam.ml.gcp import naturallanguageml as nlp +except ImportError: + nlp = None + # Quiet some pylint warnings that happen because of the somewhat special # format for the code snippets. # pylint:disable=invalid-name @@ -1520,3 +1528,86 @@ def bigqueryio_deadletter(): # [END BigQueryIODeadLetter] return result + + +def extract_sentiments(response): + # [START nlp_extract_sentiments] + return { + 'sentences': [{ + sentence.text.content: sentence.sentiment.score + } for sentence in response.sentences], + 'document_sentiment': response.document_sentiment.score, + } + # [END nlp_extract_sentiments] + + +def extract_entities(response): + # [START nlp_extract_entities] + return [{ + 'name': entity.name, + 'type': nlp.enums.Entity.Type(entity.type).name, + } for entity in response.entities] + # [END nlp_extract_entities] + + +def analyze_dependency_tree(response): + # [START analyze_dependency_tree] + from collections import defaultdict + adjacency_lists = [] + + index = 0 + for sentence in response.sentences: + adjacency_list = defaultdict(list) + sentence_begin = sentence.text.begin_offset + sentence_end = sentence_begin + len(sentence.text.content) - 1 + + while index < len(response.tokens) and \ + response.tokens[index].text.begin_offset <= sentence_end: + token = response.tokens[index] + head_token_index = token.dependency_edge.head_token_index + head_token_text = response.tokens[head_token_index].text.content + adjacency_list[head_token_text].append(token.text.content) + index += 1 + adjacency_lists.append(adjacency_list) + # [END analyze_dependency_tree] + + return adjacency_lists + + +def nlp_analyze_text(): + # [START nlp_analyze_text] + features = nlp.types.AnnotateTextRequest.Features( + extract_entities=True, + extract_document_sentiment=True, + extract_entity_sentiment=True, + extract_syntax=True, + ) + + with beam.Pipeline() as p: + responses = ( + p + | beam.Create([ + 'My experience so far has been fantastic! ' + 'I\'d really recommend this product.' + ]) + | beam.Map(lambda x: nlp.Document(x, type='PLAIN_TEXT')) + | nlp.AnnotateText(features)) + + _ = ( + responses + | beam.Map(extract_sentiments) + | 'Parse sentiments to JSON' >> beam.Map(json.dumps) + | 'Write sentiments' >> beam.io.WriteToText('sentiments.txt')) + + _ = ( + responses + | beam.Map(extract_entities) + | 'Parse entities to JSON' >> beam.Map(json.dumps) + | 'Write entities' >> beam.io.WriteToText('entities.txt')) + + _ = ( + responses + | beam.Map(analyze_dependency_tree) + | 'Parse adjacency list to JSON' >> beam.Map(json.dumps) + | 'Write adjacency list' >> beam.io.WriteToText('adjancency_list.txt')) + # [END nlp_analyze_text] diff --git a/website/www/site/content/en/documentation/patterns/ai-platform.md b/website/www/site/content/en/documentation/patterns/ai-platform.md new file mode 100644 index 000000000000..d5509679b26b --- /dev/null +++ b/website/www/site/content/en/documentation/patterns/ai-platform.md @@ -0,0 +1,159 @@ +--- +title: "AI Platform integration patterns" +--- + + +# AI Platform integration patterns + +This page describes common patterns in pipelines with Google Cloud AI Platform transforms. + +{{< language-switcher java py >}} + +## Analysing the structure and meaning of text + +This section shows how to use [Google Cloud Natural Language API](https://cloud.google.com/natural-language) to perform text analysis. + +Beam provides a PTransform called [AnnotateText](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.gcp.naturallanguageml.html#apache_beam.ml.gcp.naturallanguageml.AnnotateText). The transform takes a PCollection of type [Document](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.gcp.naturallanguageml.html#apache_beam.ml.gcp.naturallanguageml.Document). Each Document object contains various information about text. This includes the content, whether it is a plain text or HTML, an optional language hint and other settings. +`AnnotateText` produces response object of type `AnnotateTextResponse` returned from the API. `AnnotateTextResponse` is a protobuf message which contains a lot of attributes, some of which are complex structures. + +Here is an example of a pipeline that creates in-memory PCollection of strings, changes each string to Document object and invokes Natural Language API. Then, for each response object, a function is called to extract certain results of analysis. + +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" nlp_analyze_text >}} +{{< /highlight >}} + +{{< highlight java >}} +// Java examples will be available on Beam 2.23 release. +{{< /highlight >}} + + +### Extracting sentiments + +This is a part of response object returned from the API. Sentence-level sentiments can be found in `sentences` attribute. `sentences` behaves like a standard Python sequence, therefore all core language features (like iteration or slicing) will work. Overall sentiment can be found in `document_sentiment` attribute. + +``` +sentences { + text { + content: "My experience so far has been fantastic!" + } + sentiment { + magnitude: 0.8999999761581421 + score: 0.8999999761581421 + } +} +sentences { + text { + content: "I\'d really recommend this product." + begin_offset: 41 + } + sentiment { + magnitude: 0.8999999761581421 + score: 0.8999999761581421 + } +} + +...many lines omitted + +document_sentiment { + magnitude: 1.899999976158142 + score: 0.8999999761581421 +} +``` + +The function for extracting information about sentence-level and document-level sentiments is shown in the next code snippet. + +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" nlp_extract_sentiments >}} +{{< /highlight >}} + +{{< highlight java >}} +// Java examples will be available on Beam 2.23 release. +{{< /highlight >}} + +The snippet loops over `sentences` and, for each sentence, extracts the sentiment score. + +The output is: + +``` +{"sentences": [{"My experience so far has been fantastic!": 0.8999999761581421}, {"I'd really recommend this product.": 0.8999999761581421}], "document_sentiment": 0.8999999761581421} +``` + +### Extracting entities + +The next function inspects the response for entities and returns the names and the types of those entities. + +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" nlp_extract_entities >}} +{{< /highlight >}} + +{{< highlight java >}} +// Java examples will be available on Beam 2.23 release. +{{< /highlight >}} + +Entities can be found in `entities` attribute. Just like before, `entities` is a sequence, that's why list comprehension is a viable choice. The most tricky part is interpreting the types of entities. Natural Language API defines entity types as enum. In a response object, entity types are returned as integers. That's why a user has to instantiate `naturallanguageml.enums.Entity.Type` to access a human-readable name. + +The output is: + +``` +[{"name": "experience", "type": "OTHER"}, {"name": "product", "type": "CONSUMER_GOOD"}] +``` + +### Accessing sentence dependency tree + +The following code loops over the sentences and, for each sentence, builds an adjacency list that represents a dependency tree. For more information on what dependency tree is, see [Morphology & Dependency Trees](https://cloud.google.com/natural-language/docs/morphology#dependency_trees). + +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" analyze_dependency_tree >}} +{{< /highlight >}} + +{{< highlight java >}} +// Java examples will be available on Beam 2.23 release. +{{< /highlight >}} + +The output is below. For better readability, indexes are replaced by text which they refer to: + +``` +[ + { + "experience": [ + "My" + ], + "been": [ + "experience", + "far", + "has", + "been", + "fantastic", + "!" + ], + "far": [ + "so" + ] + }, + { + "recommend": [ + "I", + "'d", + "really", + "recommend", + "product", + "." + ], + "product": [ + "this" + ] + } +] +``` diff --git a/website/www/site/content/en/documentation/patterns/overview.md b/website/www/site/content/en/documentation/patterns/overview.md index 561c8f00187c..225ef8429d2b 100644 --- a/website/www/site/content/en/documentation/patterns/overview.md +++ b/website/www/site/content/en/documentation/patterns/overview.md @@ -38,6 +38,9 @@ Pipeline patterns demonstrate common Beam use cases. Pipeline patterns are based **BigQuery patterns** - Patterns for BigQueryIO * [Google BigQuery patterns](/documentation/patterns/bigqueryio/#google-bigquery-patterns) +**AI Platform integration patterns** - Patterns for using Google Cloud AI Platform transforms +* [Analysing the structure and meaning of text](/documentation/patterns/ai-platform/#analysing-the-structure-and-meaning-of-text) + ## Contributing a pattern To contribute a new pipeline pattern, create an issue with the [`pipeline-patterns` label](https://issues.apache.org/jira/browse/BEAM-7449?jql=labels%20%3D%20pipeline-patterns) and add details to the issue description. See [Get started contributing](/contribute/) for more information. diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 68e541072281..b83774d34f06 100644 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -174,17 +174,17 @@

  • @@ -212,7 +212,7 @@
  • - +
  • Java @@ -224,18 +224,18 @@
  • @@ -273,7 +273,7 @@
  • - + @@ -289,6 +289,7 @@
  • Custom I/O
  • Custom windows
  • BigQueryIO
  • +
  • AI Platform
  • From e9837feb508db7c2a2c1b2d1c8da1135caca3ecb Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Mon, 1 Jun 2020 11:56:02 +0200 Subject: [PATCH 130/151] [BEAM-9421] Website section that describes getting predictions using AI Platform Prediciton (#11075) --- .../en/documentation/patterns/ai-platform.md | 57 +++++++++++++++++++ .../en/documentation/patterns/overview.md | 1 + 2 files changed, 58 insertions(+) diff --git a/website/www/site/content/en/documentation/patterns/ai-platform.md b/website/www/site/content/en/documentation/patterns/ai-platform.md index d5509679b26b..905f89584447 100644 --- a/website/www/site/content/en/documentation/patterns/ai-platform.md +++ b/website/www/site/content/en/documentation/patterns/ai-platform.md @@ -157,3 +157,60 @@ The output is below. For better readability, indexes are replaced by text which } ] ``` + +## Getting predictions + +This section shows how to use [Google Cloud AI Platform Prediction](https://cloud.google.com/ai-platform/prediction/docs/overview) to make predictions about new data from a cloud-hosted machine learning model. + +[tfx_bsl](https://github.com/tensorflow/tfx-bsl) is a library with a Beam PTransform called `RunInference`. `RunInference` is able to perform an inference that can use an external service endpoint for receiving data. When using a service endpoint, the transform takes a PCollection of type `tf.train.Example` and, for every batch of elements, sends a request to AI Platform Prediction. The size of a batch is automatically computed. For more details on how Beam finds the best batch size, refer to a docstring for [BatchElements](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.util.html?highlight=batchelements#apache_beam.transforms.util.BatchElements). Currently, the transform does not support using `tf.train.SequenceExample` as input, but the work is in progress. + + The transform produces a PCollection of type `PredictionLog`, which contains predictions. + +Before getting started, deploy a TensorFlow model to AI Platform Prediction. The cloud service manages the infrastructure needed to handle prediction requests in both efficient and scalable way. Do note that only TensorFlow models are supported by the transform. For more information, see [Exporting a SavedModel for prediction](https://cloud.google.com/ai-platform/prediction/docs/exporting-savedmodel-for-prediction). + +Once a machine learning model is deployed, prepare a list of instances to get predictions for. To send binary data, make sure that the name of an input ends in `_bytes`. This will base64-encode data before sending a request. + +### Example +Here is an example of a pipeline that reads input instances from the file, converts JSON objects to `tf.train.Example` objects and sends data to AI Platform Prediction. The content of a file can look like this: + +``` +{"input": "the quick brown"} +{"input": "la bruja le"} +``` + +The example creates `tf.train.BytesList` instances, thus it expects byte-like strings as input. However, other data types, like `tf.train.FloatList` and `tf.train.Int64List`, are also supported by the transform. + +Here is the code: + +{{< highlight py >}} +import json + +import apache_beam as beam + +import tensorflow as tf +from tfx_bsl.beam.run_inference import RunInference +from tfx_bsl.proto import model_spec_pb2 + +def convert_json_to_tf_example(json_obj): + samples = json.loads(json_obj) + for name, text in samples.items(): + value = tf.train.Feature(bytes_list=tf.train.BytesList( + value=[text.encode('utf-8')])) + feature = {name: value} + return tf.train.Example(features=tf.train.Features(feature=feature)) + +with beam.Pipeline() as p: + _ = (p + | beam.io.ReadFromText('gs://my-bucket/samples.json') + | beam.Map(convert_json_to_tf_example) + | RunInference( + model_spec_pb2.InferenceEndpoint( + model_endpoint_spec=model_spec_pb2.AIPlatformPredictionModelSpec( + project_id='my-project-id', + model_name='my-model-name', + version_name='my-model-version')))) +{{< /highlight >}} + +{{< highlight java >}} +// Getting predictions is not yet available for Java. [BEAM-9501] +{{< /highlight >}} diff --git a/website/www/site/content/en/documentation/patterns/overview.md b/website/www/site/content/en/documentation/patterns/overview.md index 225ef8429d2b..3bc682d2ab97 100644 --- a/website/www/site/content/en/documentation/patterns/overview.md +++ b/website/www/site/content/en/documentation/patterns/overview.md @@ -40,6 +40,7 @@ Pipeline patterns demonstrate common Beam use cases. Pipeline patterns are based **AI Platform integration patterns** - Patterns for using Google Cloud AI Platform transforms * [Analysing the structure and meaning of text](/documentation/patterns/ai-platform/#analysing-the-structure-and-meaning-of-text) +* [Getting predictions](/documentation/patterns/ai-platform/#getting-predictions) ## Contributing a pattern From a1031c8151ec117742a3d496309327d5c1a0b78e Mon Sep 17 00:00:00 2001 From: Piotr Szuberski Date: Mon, 1 Jun 2020 17:47:14 +0200 Subject: [PATCH 131/151] [BEAM-10162, BEAM-10163] Change PubSubIO and BigQueryIO job names; update README.md --- .test-infra/jenkins/README.md | 3 +++ .../jenkins/job_PerformanceTests_BigQueryIO_Python.groovy | 8 ++++---- .../jenkins/job_PerformanceTests_PubsubIO_Python.groovy | 4 ++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/.test-infra/jenkins/README.md b/.test-infra/jenkins/README.md index 18aaa40cf4fd..67c9e446f392 100644 --- a/.test-infra/jenkins/README.md +++ b/.test-infra/jenkins/README.md @@ -94,6 +94,8 @@ Beam Jenkins overview page: [link](https://builds.apache.org/view/A-D/view/Beam/ |------|------|-------------------|-------------| | beam_PerformanceTests_Analysis | [cron](https://builds.apache.org/job/beam_PerformanceTests_Analysis/) | `Run Performance Tests Analysis` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_Analysis/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_Analysis) | | beam_PerformanceTests_AvroIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT/), [hdfs_cron](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT_HDFS/) | `Run Java AvroIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT) [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT_HDFS/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_AvroIOIT_HDFS) | +| beam_PerformanceTests_BiqQueryIO_Read_Python | [cron](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Read_Python/) | `Run BigQueryIO Read Performance Test Python` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Read_Python/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Read_Python) | +| beam_PerformanceTests_BiqQueryIO_Write_Python_Batch | [cron](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch/) | `Run BigQueryIO Write Performance Test Python Batch` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch) | | beam_PerformanceTests_Compressed_TextIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT/), [hdfs_cron](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT_HDFS/) | `Run Java CompressedTextIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT) [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT_HDFS/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_Compressed_TextIOIT_HDFS) | | beam_PerformanceTests_HadoopFormat | [cron](https://builds.apache.org/job/beam_PerformanceTests_HadoopFormat/) | `Run Java HadoopFormatIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_HadoopFormat/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_HadoopFormat) | | beam_PerformanceTests_JDBC | [cron](https://builds.apache.org/job/beam_PerformanceTests_JDBC/) | `Run Java JdbcIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_JDBC/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_JDBC) | @@ -101,6 +103,7 @@ Beam Jenkins overview page: [link](https://builds.apache.org/view/A-D/view/Beam/ | beam_PerformanceTests_ManyFiles_TextIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT/), [hdfs_cron](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS/) | `Run Java ManyFilesTextIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT) [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS) | | beam_PerformanceTests_MongoDBIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_MongoDBIO_IT/) | `Run Java MongoDBIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_MongoDBIO_IT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_MongoDBIO_IT) | | beam_PerformanceTests_ParquetIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT/), [hdfs_cron](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT_HDFS/) | `Run Java ParquetIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT) [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT_HDFS/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_ParquetIOIT_HDFS) | +| beam_PerformanceTests_PubsubIOIT_Python_Streaming | [cron](https://builds.apache.org/job/beam_PerformanceTests_PubsubIOIT_Python_Streaming/) | `Run PubsubIO Performance Test Python` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_PubsubIOIT_Python_Streaming/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_PubsubIOIT_Python_Streaming) | | beam_PerformanceTests_Spark | [cron](https://builds.apache.org/job/beam_PerformanceTests_Spark/) | `Run Spark Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_Spark/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_Spark) | | beam_PerformanceTests_TFRecordIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_TFRecordIOIT/) | `Run Java TFRecordIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_TFRecordIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_TFRecordIOIT) | | beam_PerformanceTests_TextIOIT | [cron](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT/), [hdfs_cron](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT_HDFS/) | `Run Java TextIO Performance Test` | [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT) [![Build Status](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT_HDFS/badge/icon)](https://builds.apache.org/job/beam_PerformanceTests_TextIOIT_HDFS) | diff --git a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy index c8fcc6825344..56dc0c894192 100644 --- a/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy +++ b/.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy @@ -82,7 +82,7 @@ def executeJob = { scope, testConfig -> } PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_BiqQueryIO_Read_Performance_Test_Python', + 'beam_PerformanceTests_BiqQueryIO_Read_Python', 'Run BigQueryIO Read Performance Test Python', 'BigQueryIO Read Performance Test Python', this @@ -90,12 +90,12 @@ PhraseTriggeringPostCommitBuilder.postCommitJob( executeJob(delegate, bqio_read_test) } -CronJobBuilder.cronJob('beam_BiqQueryIO_Read_Performance_Test_Python', 'H 15 * * *', this) { +CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Read_Python', 'H 15 * * *', this) { executeJob(delegate, bqio_read_test) } PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_BiqQueryIO_Write_Performance_Test_Python_Batch', + 'beam_PerformanceTests_BiqQueryIO_Write_Python_Batch', 'Run BigQueryIO Write Performance Test Python Batch', 'BigQueryIO Write Performance Test Python Batch', this @@ -103,6 +103,6 @@ PhraseTriggeringPostCommitBuilder.postCommitJob( executeJob(delegate, bqio_write_test) } -CronJobBuilder.cronJob('beam_BiqQueryIO_Write_Performance_Test_Python_Batch', 'H 15 * * *', this) { +CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Write_Python_Batch', 'H 15 * * *', this) { executeJob(delegate, bqio_write_test) } diff --git a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy b/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy index 96361c007920..f08e53ec765d 100644 --- a/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy +++ b/.test-infra/jenkins/job_PerformanceTests_PubsubIO_Python.groovy @@ -62,7 +62,7 @@ def executeJob = { scope, testConfig -> } PhraseTriggeringPostCommitBuilder.postCommitJob( - 'beam_PubsubIO_Performance_Test_Python', + 'beam_PerformanceTests_PubsubIOIT_Python_Streaming', 'Run PubsubIO Performance Test Python', 'PubsubIO Performance Test Python', this @@ -70,6 +70,6 @@ PhraseTriggeringPostCommitBuilder.postCommitJob( executeJob(delegate, psio_test) } -CronJobBuilder.cronJob('beam_PubsubIO_Performance_Test_Python', 'H 15 * * *', this) { +CronJobBuilder.cronJob('beam_PerformanceTests_PubsubIOIT_Python_Streaming', 'H 15 * * *', this) { executeJob(delegate, psio_test) } From 194f981cca8de97a9380b39eec67a98b13bddc71 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Mon, 1 Jun 2020 08:49:37 -0700 Subject: [PATCH 132/151] [BEAM-10051] Scope instruction data. (#11864) * [BEAM-10051] Scope instruction data. --- .../pkg/beam/core/runtime/harness/datamgr.go | 128 ++++++++++++++---- .../beam/core/runtime/harness/datamgr_test.go | 73 ++++++++-- 2 files changed, 167 insertions(+), 34 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 3fb9401dd017..08e62f2233c3 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -38,8 +38,6 @@ type ScopedDataManager struct { mgr *DataChannelManager instID instructionID - // TODO(herohde) 7/20/2018: capture and force close open reads/writes. However, - // we would need the underlying Close to be idempotent or a separate method. closed bool mu sync.Mutex } @@ -82,9 +80,10 @@ func (s *ScopedDataManager) open(ctx context.Context, port exec.Port) (*DataChan // Close prevents new IO for this instruction. func (s *ScopedDataManager) Close() error { s.mu.Lock() + defer s.mu.Unlock() s.closed = true + s.mgr.closeInstruction(s.instID) s.mgr = nil - s.mu.Unlock() return nil } @@ -125,6 +124,14 @@ func (m *DataChannelManager) Open(ctx context.Context, port exec.Port) (*DataCha return ch, nil } +func (m *DataChannelManager) closeInstruction(instID instructionID) { + m.mu.Lock() + defer m.mu.Unlock() + for _, ch := range m.ports { + ch.removeInstruction(instID) + } +} + // clientID identifies a client of a connected channel. type clientID struct { ptransformID string @@ -148,8 +155,13 @@ type DataChannel struct { id string client dataClient - writers map[clientID]*dataWriter - readers map[clientID]*dataReader + writers map[instructionID]map[string]*dataWriter + readers map[instructionID]map[string]*dataReader + + // recently terminated instructions + endedInstructions map[instructionID]struct{} + rmQueue []instructionID + // readErr indicates a client.Recv error and is used to prevent new readers. readErr error @@ -178,11 +190,12 @@ func newDataChannel(ctx context.Context, port exec.Port) (*DataChannel, error) { func makeDataChannel(ctx context.Context, id string, client dataClient, cancelFn context.CancelFunc) *DataChannel { ret := &DataChannel{ - id: id, - client: client, - writers: make(map[clientID]*dataWriter), - readers: make(map[clientID]*dataReader), - cancelFn: cancelFn, + id: id, + client: client, + writers: make(map[instructionID]map[string]*dataWriter), + readers: make(map[instructionID]map[string]*dataReader), + endedInstructions: make(map[instructionID]struct{}), + cancelFn: cancelFn, } go ret.read(ctx) @@ -227,14 +240,16 @@ func (c *DataChannel) read(ctx context.Context) { // close the r.buf channels twice, or send on a closed channel. // Any other approach is racy, and may cause one of the above // panics. - for _, r := range c.readers { - log.Errorf(ctx, "DataChannel.read %v reader %v closing due to error on channel", c.id, r.id) - if !r.completed { - r.completed = true - r.err = err - close(r.buf) + for _, m := range c.readers { + for _, r := range m { + log.Errorf(ctx, "DataChannel.read %v reader %v closing due to error on channel", c.id, r.id) + if !r.completed { + r.completed = true + r.err = err + close(r.buf) + } + delete(cache, r.id) } - delete(cache, r.id) } c.terminateStreamOnError(err) c.mu.Unlock() @@ -319,31 +334,95 @@ func (r *errReader) Close() error { // makeReader creates a dataReader. It expects to be called while c.mu is held. func (c *DataChannel) makeReader(ctx context.Context, id clientID) *dataReader { - if r, ok := c.readers[id]; ok { + var m map[string]*dataReader + var ok bool + if m, ok = c.readers[id.instID]; !ok { + m = make(map[string]*dataReader) + c.readers[id.instID] = m + } + + if r, ok := m[id.ptransformID]; ok { return r } r := &dataReader{id: id, buf: make(chan []byte, bufElements), done: make(chan bool, 1), channel: c} - c.readers[id] = r + + // Just in case initial data for an instruction arrives *after* an instructon has ended. + // eg. it was blocked by another reader being slow, or the other instruction failed. + // So we provide a pre-completed reader, and do not cache it, as there's no further cleanup for it. + if _, ok := c.endedInstructions[id.instID]; ok { + r.completed = true + close(r.buf) + r.err = io.EOF // In case of any actual data readers, so they terminate without error. + return r + } + + m[id.ptransformID] = r return r } func (c *DataChannel) removeReader(id clientID) { c.mu.Lock() - delete(c.readers, id) + if m, ok := c.readers[id.instID]; ok { + delete(m, id.ptransformID) + } c.mu.Unlock() } +const endedInstructionCap = 32 + +// removeInstruction closes all readers and writers registered for the instruction +// and deletes this instruction from the channel's reader and writer maps. +func (c *DataChannel) removeInstruction(instID instructionID) { + c.mu.Lock() + + // We don't want to leak memory, so cap the endedInstructions list. + if len(c.rmQueue) >= endedInstructionCap { + toRemove := c.rmQueue[0] + c.rmQueue = c.rmQueue[1:] + delete(c.endedInstructions, toRemove) + } + c.endedInstructions[instID] = struct{}{} + c.rmQueue = append(c.rmQueue, instID) + + rs := c.readers[instID] + ws := c.writers[instID] + + // Prevent other users while we iterate. + delete(c.readers, instID) + delete(c.writers, instID) + c.mu.Unlock() + + // Close grabs the channel lock, so this must be outside the critical section. + for _, r := range rs { + r.Close() + } + for _, w := range ws { + w.Close() + } +} + func (c *DataChannel) makeWriter(ctx context.Context, id clientID) *dataWriter { c.mu.Lock() defer c.mu.Unlock() - if w, ok := c.writers[id]; ok { + var m map[string]*dataWriter + var ok bool + if m, ok = c.writers[id.instID]; !ok { + m = make(map[string]*dataWriter) + c.writers[id.instID] = m + } + + if w, ok := m[id.ptransformID]; ok { return w } + // We don't check for ended instructions for writers, as writers + // can only be created if an instruction is in scope, and aren't + // runner or user directed. + w := &dataWriter{ch: c, id: id} - c.writers[id] = w + m[id.ptransformID] = w return w } @@ -389,9 +468,6 @@ func (r *dataReader) Read(buf []byte) (int, error) { return n, nil } -// TODO(herohde) 7/20/2018: we should probably either not be tracking writers or -// make dataWriter threadsafe. Either case is likely a corruption generator. - type dataWriter struct { buf []byte @@ -431,7 +507,7 @@ func (w *dataWriter) Close() error { // Now acquire the locks since we're sending. w.ch.mu.Lock() defer w.ch.mu.Unlock() - delete(w.ch.writers, w.id) + delete(w.ch.writers[w.id.instID], w.id.ptransformID) msg := &fnpb.Elements{ Data: []*fnpb.Elements_Data{ { diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index 5601c3823fd4..31172e8a04b0 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -22,6 +22,7 @@ import ( "io/ioutil" "log" "strings" + "sync" "testing" "time" @@ -38,9 +39,14 @@ type fakeDataClient struct { calls int err error skipFirstError bool + + blocked sync.Mutex // Prevent data from being read by the gotourtinr. } func (f *fakeDataClient) Recv() (*fnpb.Elements, error) { + f.blocked.Lock() + defer f.blocked.Unlock() + f.calls++ data := []byte{1, 2, 3, 4, 1, 2, 3, 4} elemData := fnpb.Elements_Data{ @@ -130,6 +136,12 @@ func TestDataChannelTerminate_dataReader(t *testing.T) { // Set the 2nd Recv call to have an error. client.err = expectedError }, + }, { + name: "onInstructionEnd", + expectedError: io.EOF, + caseFn: func(t *testing.T, r io.ReadCloser, client *fakeDataClient, c *DataChannel) { + c.removeInstruction("inst_ref") + }, }, } for _, test := range tests { @@ -170,7 +182,46 @@ func TestDataChannelTerminate_dataReader(t *testing.T) { } }) } +} + +func TestDataChannelRemoveInstruction_dataAfterClose(t *testing.T) { + done := make(chan bool, 1) + client := &fakeDataClient{t: t, done: done} + client.blocked.Lock() + + ctx, cancelFn := context.WithCancel(context.Background()) + c := makeDataChannel(ctx, "id", client, cancelFn) + c.removeInstruction("inst_ref") + + client.blocked.Unlock() + + r := c.OpenRead(ctx, "ptr", "inst_ref") + + dr := r.(*dataReader) + if !dr.completed || dr.err != io.EOF { + t.Errorf("Expected a closed reader, but was still open: completed: %v, err: %v", dr.completed, dr.err) + } + n, err := r.Read(make([]byte, 4)) + if err != io.EOF { + t.Errorf("Unexpected error from read: %v, read %d bytes.", err, n) + } +} + +func TestDataChannelRemoveInstruction_limitInstructionCap(t *testing.T) { + done := make(chan bool, 1) + client := &fakeDataClient{t: t, done: done} + ctx, cancelFn := context.WithCancel(context.Background()) + c := makeDataChannel(ctx, "id", client, cancelFn) + + for i := 0; i < endedInstructionCap+10; i++ { + instID := instructionID(fmt.Sprintf("inst_ref%d", i)) + c.OpenRead(ctx, "ptr", instID) + c.removeInstruction(instID) + } + if got, want := len(c.endedInstructions), endedInstructionCap; got != want { + t.Errorf("unexpected len(endedInstructions) got %v, want %v,", got, want) + } } func TestDataChannelTerminate_Writes(t *testing.T) { @@ -179,27 +230,34 @@ func TestDataChannelTerminate_Writes(t *testing.T) { expectedError := fmt.Errorf("EXPECTED ERROR") + instID := instructionID("inst_ref") tests := []struct { name string - caseFn func(t *testing.T, w io.WriteCloser, client *fakeDataClient) error + caseFn func(t *testing.T, w io.WriteCloser, client *fakeDataClient, c *DataChannel) error }{ { name: "onClose_Flush", - caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient) error { + caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient, c *DataChannel) error { return w.Close() }, }, { name: "onClose_Sentinel", - caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient) error { + caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient, c *DataChannel) error { client.skipFirstError = true return w.Close() }, }, { name: "onWrite", - caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient) error { + caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient, c *DataChannel) error { _, err := w.Write([]byte{'d', 'o', 'n', 'e'}) return err }, + }, { + name: "onInstructionEnd", + caseFn: func(t *testing.T, w io.WriteCloser, client *fakeDataClient, c *DataChannel) error { + c.removeInstruction(instID) + return expectedError + }, }, } for _, test := range tests { @@ -209,7 +267,7 @@ func TestDataChannelTerminate_Writes(t *testing.T) { ctx, cancelFn := context.WithCancel(context.Background()) c := makeDataChannel(ctx, "id", client, cancelFn) - w := c.OpenWrite(ctx, "ptr", "inst_ref") + w := c.OpenWrite(ctx, "ptr", instID) msg := []byte{'b', 'y', 't', 'e'} var bufSize int @@ -221,7 +279,7 @@ func TestDataChannelTerminate_Writes(t *testing.T) { } } - err := test.caseFn(t, w, client) + err := test.caseFn(t, w, client, c) if got, want := err, expectedError; err == nil || !strings.Contains(err.Error(), expectedError.Error()) { t.Errorf("Unexpected error: got %v, want %v", got, want) @@ -229,7 +287,7 @@ func TestDataChannelTerminate_Writes(t *testing.T) { // Verify that new readers return the same error for writes after stream termination. // TODO(lostluck) 2019.11.26: use the the go 1.13 errors package to check this rather // than a strings.Contains check once testing infrastructure can use go 1.13. - if n, err := c.OpenWrite(ctx, "ptr", "inst_ref").Write(msg); err != nil && !strings.Contains(err.Error(), expectedError.Error()) { + if n, err := c.OpenWrite(ctx, "ptr", instID).Write(msg); err != nil && !strings.Contains(err.Error(), expectedError.Error()) { t.Errorf("Unexpected error from write: got %v, want, %v read %d bytes.", err, expectedError, n) } select { @@ -240,5 +298,4 @@ func TestDataChannelTerminate_Writes(t *testing.T) { } }) } - } From e0217d1efa00d37ef66db040d06da610ea9e8c99 Mon Sep 17 00:00:00 2001 From: Damon Douglas Date: Mon, 1 Jun 2020 08:50:22 -0700 Subject: [PATCH 133/151] [BEAM-9679] Add Combine/Simple Function to Core Transforms (#11866) --- .../Combine/Simple Function/cmd/main.go | 43 +++ .../Combine/Simple Function/go.mod | 27 ++ .../Combine/Simple Function/go.sum | 310 ++++++++++++++++++ .../Combine/Simple Function/pkg/task/task.go | 26 ++ .../Combine/Simple Function/task-info.yaml | 35 ++ .../Simple Function/task-remote-info.yaml | 2 + .../Combine/Simple Function/task.md | 43 +++ .../Combine/Simple Function/test/task_test.go | 44 +++ .../Core Transforms/Combine/lesson-info.yaml | 21 ++ .../Combine/lesson-remote-info.yaml | 3 + .../Flatten/lesson-remote-info.yaml | 2 +- .../go/Core Transforms/section-info.yaml | 1 + learning/katas/go/course-remote-info.yaml | 2 +- 13 files changed, 557 insertions(+), 2 deletions(-) create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/cmd/main.go create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/go.mod create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/go.sum create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/pkg/task/task.go create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/task-info.yaml create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/task-remote-info.yaml create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/task.md create mode 100644 learning/katas/go/Core Transforms/Combine/Simple Function/test/task_test.go create mode 100644 learning/katas/go/Core Transforms/Combine/lesson-info.yaml create mode 100644 learning/katas/go/Core Transforms/Combine/lesson-remote-info.yaml diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/cmd/main.go b/learning/katas/go/Core Transforms/Combine/Simple Function/cmd/main.go new file mode 100644 index 000000000000..eb413972a7b2 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/cmd/main.go @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "context" + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/log" + "github.com/apache/beam/sdks/go/pkg/beam/x/beamx" + "github.com/apache/beam/sdks/go/pkg/beam/x/debug" + "simple_function/pkg/task" +) + +func main() { + ctx := context.Background() + + p, s := beam.NewPipelineWithRoot() + + input := beam.Create(s, 10, 30, 50, 70, 90) + + output := task.ApplyTransform(s, input) + + debug.Print(s, output) + + err := beamx.Run(ctx, p) + + if err != nil { + log.Exitf(context.Background(), "Failed to execute job: %v", err) + } +} diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/go.mod b/learning/katas/go/Core Transforms/Combine/Simple Function/go.mod new file mode 100644 index 000000000000..f8cdb28c1a10 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/go.mod @@ -0,0 +1,27 @@ +// 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. + +module simple_function + +go 1.14 + +require ( + github.com/apache/beam v2.21.0+incompatible + github.com/golang/protobuf v1.4.2 // indirect + github.com/google/go-cmp v0.4.1 // indirect + github.com/googleapis/gax-go v1.0.3 // indirect + google.golang.org/api v0.25.0 // indirect + google.golang.org/grpc v1.29.1 // indirect +) diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/go.sum b/learning/katas/go/Core Transforms/Combine/Simple Function/go.sum new file mode 100644 index 000000000000..00ef75933562 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/go.sum @@ -0,0 +1,310 @@ +cloud.google.com/go v0.26.0 h1:e0WKqKTd5BnrG8aKH3J3h+QvEIQtSUcf2n5UZ5ZgLtQ= +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.56.0 h1:WRz29PgAsVEyPSDHyk+0fpEkwEFyfhHn+JbksT6gIL4= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0 h1:UDpwYIwla4jHGzZJaEJYx1tOejbgSoNqsAfHAUYe2r8= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/apache/beam v2.21.0+incompatible h1:ETfDaxjAOSQqy44CyaxqrSW5omoLzmEZmThJEtSLHBI= +github.com/apache/beam v2.21.0+incompatible/go.mod h1:/8NX3Qi8vGstDLLaeaU7+lzVEu/ACaQhYjeefzQ0y1o= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1 h1:/exdXoGamhu5ONeUJH0deniYLWYvQwW66yvlfiiKTu0= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/googleapis/gax-go v1.0.3 h1:9dMLqhaibYONnDRcnHdUs9P8Mw64jLlZTYlDe3leBtQ= +github.com/googleapis/gax-go v1.0.3/go.mod h1:QyXYajJFdARxGzjwUfbDFIse7Spkw81SJ4LrBJXtlQ8= +github.com/googleapis/gax-go/v2 v2.0.2/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190221220918-438050ddec5e/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be h1:vEDujvNQGv4jgYKudGeI/+DAX4Jffq6hpD55MmoEvKs= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.25.0 h1:LodzhlzZEUfhXzNUMIfVlf9Gr6Ua5MMtoFWh7+f47qA= +google.golang.org/api v0.25.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940 h1:MRHtG0U6SnaUb+s+LhNE1qt1FQ1wlhqr5E4usBKC0uA= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/pkg/task/task.go b/learning/katas/go/Core Transforms/Combine/Simple Function/pkg/task/task.go new file mode 100644 index 000000000000..afe5abbfa164 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/pkg/task/task.go @@ -0,0 +1,26 @@ +// 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 task + +import ( + "github.com/apache/beam/sdks/go/pkg/beam" +) + +func ApplyTransform(s beam.Scope, input beam.PCollection) beam.PCollection { + return beam.Combine(s, func(sum, elem int) int { + return sum + elem + }, input) +} diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/task-info.yaml b/learning/katas/go/Core Transforms/Combine/Simple Function/task-info.yaml new file mode 100644 index 000000000000..1b3b8e5fd670 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/task-info.yaml @@ -0,0 +1,35 @@ +# +# 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. +# + +type: edu +files: +- name: test/task_test.go + visible: false +- name: cmd/main.go + visible: true +- name: go.mod + visible: false +- name: pkg/task/task.go + visible: true + placeholders: + - offset: 949 + length: 72 + placeholder_text: TODO() +- name: go.sum + visible: false diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/task-remote-info.yaml b/learning/katas/go/Core Transforms/Combine/Simple Function/task-remote-info.yaml new file mode 100644 index 000000000000..098280d38d3c --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/task-remote-info.yaml @@ -0,0 +1,2 @@ +id: 1347871 +update_date: Mon, 01 Jun 2020 14:35:52 UTC diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/task.md b/learning/katas/go/Core Transforms/Combine/Simple Function/task.md new file mode 100644 index 000000000000..ea0cc8c52760 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/task.md @@ -0,0 +1,43 @@ + + +# Combine - Simple Function + +Combine is a Beam transform for combining collections of elements or values in your data. When you +apply a Combine transform, you must provide the function that contains the logic for combining the +elements or values. The combining function should be commutative and associative, as the function +is not necessarily invoked exactly once on all values with a given key. Because the input data +(including the value collection) may be distributed across multiple workers, the combining function +might be called multiple times to perform partial combining on subsets of the value collection. + +Simple combine operations, such as sums, can usually be implemented as a simple function. + +**Kata:** Implement the summation of numbers using +[beam.Combine](https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#Combine). + +
    + Implement a combine function compatible with the + + beam.Combine method that performs the summation of a PCollection<int>. +
    + +
    + Refer to the Beam Programming Guide + + "Simple combinations using simple functions" section for more information. +
    diff --git a/learning/katas/go/Core Transforms/Combine/Simple Function/test/task_test.go b/learning/katas/go/Core Transforms/Combine/Simple Function/test/task_test.go new file mode 100644 index 000000000000..2b0ff0c3bdda --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/Simple Function/test/task_test.go @@ -0,0 +1,44 @@ +// 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 test + +import ( + "github.com/apache/beam/sdks/go/pkg/beam" + "github.com/apache/beam/sdks/go/pkg/beam/testing/passert" + "github.com/apache/beam/sdks/go/pkg/beam/testing/ptest" + "simple_function/pkg/task" + "testing" +) + +func TestApplyTransform(t *testing.T) { + p, s := beam.NewPipelineWithRoot() + tests := []struct { + input beam.PCollection + want []interface{} + }{ + { + input: beam.Create(s, 10, 30, 50, 70, 90), + want: []interface{}{250}, + }, + } + for _, tt := range tests { + got := task.ApplyTransform(s, tt.input) + passert.Equals(s, got, tt.want...) + if err := ptest.Run(p); err != nil { + t.Error(err) + } + } +} diff --git a/learning/katas/go/Core Transforms/Combine/lesson-info.yaml b/learning/katas/go/Core Transforms/Combine/lesson-info.yaml new file mode 100644 index 000000000000..698fc8cc2265 --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/lesson-info.yaml @@ -0,0 +1,21 @@ +# +# 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. +# + +content: +- Simple Function diff --git a/learning/katas/go/Core Transforms/Combine/lesson-remote-info.yaml b/learning/katas/go/Core Transforms/Combine/lesson-remote-info.yaml new file mode 100644 index 000000000000..3747bd27cf4d --- /dev/null +++ b/learning/katas/go/Core Transforms/Combine/lesson-remote-info.yaml @@ -0,0 +1,3 @@ +id: 362667 +update_date: Mon, 01 Jun 2020 14:35:48 UTC +unit: 347283 diff --git a/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml b/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml index f7ed569d95fa..a6b715b74dda 100644 --- a/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml +++ b/learning/katas/go/Core Transforms/Flatten/lesson-remote-info.yaml @@ -1,3 +1,3 @@ id: 361641 -update_date: Thu, 28 May 2020 22:33:26 UTC +update_date: Mon, 01 Jun 2020 14:35:55 UTC unit: 346170 diff --git a/learning/katas/go/Core Transforms/section-info.yaml b/learning/katas/go/Core Transforms/section-info.yaml index d228f9a30d38..8bde6d156bf8 100644 --- a/learning/katas/go/Core Transforms/section-info.yaml +++ b/learning/katas/go/Core Transforms/section-info.yaml @@ -21,4 +21,5 @@ content: - Map - GroupByKey - CoGroupByKey +- Combine - Flatten diff --git a/learning/katas/go/course-remote-info.yaml b/learning/katas/go/course-remote-info.yaml index b875f83687ed..ce7ead8cfc33 100644 --- a/learning/katas/go/course-remote-info.yaml +++ b/learning/katas/go/course-remote-info.yaml @@ -1,2 +1,2 @@ id: 70387 -update_date: Wed, 20 May 2020 06:16:18 UTC +update_date: Thu, 28 May 2020 22:34:30 UTC From 280e243dfe135ab7ed434022be78c22cdc46be8d Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Mon, 1 Jun 2020 11:58:21 -0700 Subject: [PATCH 134/151] Updates CHANGES.md to include new I/Os for Beam 2.22.0. (#11879) * Update CHANGES.md * Update CHANGES.md --- CHANGES.md | 30 +++++++++++++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index d3e706515ae5..0168d04216c6 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -47,7 +47,7 @@ * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). --> -# [2.22.0] - Unreleased +# [2.23.0] - Unreleased ## Highlights @@ -61,6 +61,34 @@ ## New Features / Improvements +* X feature added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). + +## Breaking Changes + +* X behavior was changed ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). + +## Known Issues + +* Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). + +# [2.22.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). +* New highly anticipated feature Y added to Java SDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). + +## I/Os + +* Basic Kafka read/write support for DataflowRunner (Python) ([BEAM-8019](https://issues.apache.org/jira/browse/BEAM-8019)). +* Sources and sinks for Google Healthcare APIs (Java)([BEAM-9468](https://issues.apache.org/jira/browse/BEAM-9468)). + +## New Features / Improvements + * `--workerCacheMB` flag is supported in Dataflow streaming pipeline ([BEAM-9964](https://issues.apache.org/jira/browse/BEAM-9964)) * `--direct_num_workers=0` is supported for FnApi runner. It will set the number of threads/subprocesses to number of cores of the machine executing the pipeline ([BEAM-9443](https://issues.apache.org/jira/browse/BEAM-9443)). * Python SDK now has experimental support for SqlTransform ([BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)). From 89fc35b87d5dc074d25d60a97bb96d71e04be283 Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Mon, 1 Jun 2020 15:16:19 -0700 Subject: [PATCH 135/151] [BEAM-10158] Use a shared unbounded thread pool within Beam Python. (#11867) * [BEAM-10158] Use a shared unbounded thread pool within Beam Python. * fixup! Fix lint errors * fixup! Address PR comments * fixup! fix lint --- .../runners/portability/artifact_service_test.py | 6 +++--- .../runners/portability/expansion_service_test.py | 4 ++-- .../portability/fn_api_runner/fn_runner.py | 4 ++-- .../portability/fn_api_runner/worker_handlers.py | 14 +++++++++----- .../runners/portability/local_job_service.py | 7 ++++--- .../runners/portability/portable_stager_test.py | 4 ++-- .../apache_beam/runners/worker/data_plane_test.py | 4 ++-- .../runners/worker/log_handler_test.py | 4 ++-- .../apache_beam/runners/worker/sdk_worker.py | 9 +++++---- .../apache_beam/runners/worker/sdk_worker_test.py | 4 ++-- .../runners/worker/worker_pool_main.py | 5 +++-- .../runners/worker/worker_status_test.py | 4 ++-- .../apache_beam/utils/thread_pool_executor.py | 13 +++++++++++++ .../utils/thread_pool_executor_test.py | 15 +++++++++++++++ 14 files changed, 66 insertions(+), 31 deletions(-) diff --git a/sdks/python/apache_beam/runners/portability/artifact_service_test.py b/sdks/python/apache_beam/runners/portability/artifact_service_test.py index 14993dcaabb1..6a6768ad6e0f 100644 --- a/sdks/python/apache_beam/runners/portability/artifact_service_test.py +++ b/sdks/python/apache_beam/runners/portability/artifact_service_test.py @@ -44,7 +44,7 @@ from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.runners.portability import artifact_service from apache_beam.utils import proto_utils -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor class AbstractArtifactServiceTest(unittest.TestCase): @@ -83,7 +83,7 @@ def test_basic(self): self._run_staging(self._service, self._service) def test_with_grpc(self): - server = grpc.server(UnboundedThreadPoolExecutor()) + server = grpc.server(thread_pool_executor.shared_unbounded_instance()) try: beam_artifact_api_pb2_grpc.add_LegacyArtifactStagingServiceServicer_to_server( self._service, server) @@ -225,7 +225,7 @@ def check(index): self._service, tokens[session(index)], name(index))) # pylint: disable=range-builtin-not-iterating - pool = UnboundedThreadPoolExecutor() + pool = thread_pool_executor.shared_unbounded_instance() sessions = set(pool.map(put, range(100))) tokens = dict(pool.map(commit, sessions)) # List forces materialization. diff --git a/sdks/python/apache_beam/runners/portability/expansion_service_test.py b/sdks/python/apache_beam/runners/portability/expansion_service_test.py index 6a7ab579c54d..a1b531db13be 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service_test.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service_test.py @@ -36,7 +36,7 @@ from apache_beam.runners.portability import expansion_service from apache_beam.transforms import ptransform from apache_beam.transforms.external import ImplicitSchemaPayloadBuilder -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor # This script provides an expansion service and example ptransforms for running # external transform test cases. See external_test.py for details. @@ -296,7 +296,7 @@ def main(unused_argv): '-p', '--port', type=int, help='port on which to serve the job api') options = parser.parse_args() global server - server = grpc.server(UnboundedThreadPoolExecutor()) + server = grpc.server(thread_pool_executor.shared_unbounded_instance()) beam_expansion_api_pb2_grpc.add_ExpansionServiceServicer_to_server( expansion_service.ExpansionServiceServicer( PipelineOptions( diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py index 58db57c9157f..c2147ce44182 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py @@ -64,7 +64,7 @@ from apache_beam.transforms import environments from apache_beam.utils import profiler from apache_beam.utils import proto_utils -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor if TYPE_CHECKING: from apache_beam.pipeline import Pipeline @@ -925,7 +925,7 @@ def execute(part_map_input_timers): expected_output_timers, dry_run) - with UnboundedThreadPoolExecutor() as executor: + with thread_pool_executor.shared_unbounded_instance() as executor: for result, split_result in executor.map(execute, zip(part_inputs, # pylint: disable=zip-builtin-not-iterating timer_inputs)): split_result_list += split_result diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index 6808b47823b4..6a276565f70b 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -63,7 +63,7 @@ from apache_beam.runners.worker.sdk_worker import _Future from apache_beam.runners.worker.statecache import StateCache from apache_beam.utils import proto_utils -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor # State caching is enabled in the fn_api_runner for testing, except for one # test which runs without state caching (FnApiRunnerTestWithDisabledCaching). @@ -441,7 +441,8 @@ def __init__(self, # type: (...) -> None self.state = state self.provision_info = provision_info - self.control_server = grpc.server(UnboundedThreadPoolExecutor()) + self.control_server = grpc.server( + thread_pool_executor.shared_unbounded_instance()) self.control_port = self.control_server.add_insecure_port('[::]:0') self.control_address = 'localhost:%s' % self.control_port @@ -451,11 +452,13 @@ def __init__(self, no_max_message_sizes = [("grpc.max_receive_message_length", -1), ("grpc.max_send_message_length", -1)] self.data_server = grpc.server( - UnboundedThreadPoolExecutor(), options=no_max_message_sizes) + thread_pool_executor.shared_unbounded_instance(), + options=no_max_message_sizes) self.data_port = self.data_server.add_insecure_port('[::]:0') self.state_server = grpc.server( - UnboundedThreadPoolExecutor(), options=no_max_message_sizes) + thread_pool_executor.shared_unbounded_instance(), + options=no_max_message_sizes) self.state_port = self.state_server.add_insecure_port('[::]:0') self.control_handler = BeamFnControlServicer(worker_manager) @@ -493,7 +496,8 @@ def __init__(self, GrpcStateServicer(state), self.state_server) self.logging_server = grpc.server( - UnboundedThreadPoolExecutor(), options=no_max_message_sizes) + thread_pool_executor.shared_unbounded_instance(), + options=no_max_message_sizes) self.logging_port = self.logging_server.add_insecure_port('[::]:0') beam_fn_api_pb2_grpc.add_BeamFnLoggingServicer_to_server( BasicLoggingService(), self.logging_server) diff --git a/sdks/python/apache_beam/runners/portability/local_job_service.py b/sdks/python/apache_beam/runners/portability/local_job_service.py index b99bfeb183ab..dcdc4425fcba 100644 --- a/sdks/python/apache_beam/runners/portability/local_job_service.py +++ b/sdks/python/apache_beam/runners/portability/local_job_service.py @@ -48,7 +48,7 @@ from apache_beam.runners.portability import artifact_service from apache_beam.runners.portability.fn_api_runner import fn_runner from apache_beam.runners.portability.fn_api_runner import worker_handlers -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor if TYPE_CHECKING: from google.protobuf import struct_pb2 # pylint: disable=ungrouped-imports @@ -141,7 +141,7 @@ def get_service_address(self): return 'localhost' def start_grpc_server(self, port=0): - self._server = grpc.server(UnboundedThreadPoolExecutor()) + self._server = grpc.server(thread_pool_executor.shared_unbounded_instance()) port = self._server.add_insecure_port( '%s:%d' % (self.get_bind_address(), port)) beam_job_api_pb2_grpc.add_JobServiceServicer_to_server(self, self._server) @@ -194,7 +194,8 @@ def __init__( self._worker_id = worker_id def run(self): - logging_server = grpc.server(UnboundedThreadPoolExecutor()) + logging_server = grpc.server( + thread_pool_executor.shared_unbounded_instance()) logging_port = logging_server.add_insecure_port('[::]:0') logging_server.start() logging_servicer = BeamFnLoggingServicer() diff --git a/sdks/python/apache_beam/runners/portability/portable_stager_test.py b/sdks/python/apache_beam/runners/portability/portable_stager_test.py index c7a8e1cfc674..44fb4986649e 100644 --- a/sdks/python/apache_beam/runners/portability/portable_stager_test.py +++ b/sdks/python/apache_beam/runners/portability/portable_stager_test.py @@ -36,7 +36,7 @@ from apache_beam.portability.api import beam_artifact_api_pb2 from apache_beam.portability.api import beam_artifact_api_pb2_grpc from apache_beam.runners.portability import portable_stager -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor class PortableStagerTest(unittest.TestCase): @@ -58,7 +58,7 @@ def _stage_files(self, files): describing the name of the artifacts in local temp folder and desired name in staging location. """ - server = grpc.server(UnboundedThreadPoolExecutor()) + server = grpc.server(thread_pool_executor.shared_unbounded_instance()) staging_service = TestLocalFileSystemLegacyArtifactStagingServiceServicer( self._remote_dir) beam_artifact_api_pb2_grpc.add_LegacyArtifactStagingServiceServicer_to_server( diff --git a/sdks/python/apache_beam/runners/worker/data_plane_test.py b/sdks/python/apache_beam/runners/worker/data_plane_test.py index a4922a55a3ca..4f09df350ba9 100644 --- a/sdks/python/apache_beam/runners/worker/data_plane_test.py +++ b/sdks/python/apache_beam/runners/worker/data_plane_test.py @@ -34,7 +34,7 @@ from apache_beam.runners.worker import data_plane from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor from apache_beam.testing.util import timeout -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor class DataChannelTest(unittest.TestCase): @@ -56,7 +56,7 @@ def _grpc_data_channel_test(self, time_based_flush=False): data_channel_service = \ data_servicer.get_conn_by_worker_id(worker_id) - server = grpc.server(UnboundedThreadPoolExecutor()) + server = grpc.server(thread_pool_executor.shared_unbounded_instance()) beam_fn_api_pb2_grpc.add_BeamFnDataServicer_to_server(data_servicer, server) test_port = server.add_insecure_port('[::]:0') server.start() diff --git a/sdks/python/apache_beam/runners/worker/log_handler_test.py b/sdks/python/apache_beam/runners/worker/log_handler_test.py index 4d9291d99e68..80fe543e3335 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler_test.py +++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py @@ -32,7 +32,7 @@ from apache_beam.runners.common import NameContext from apache_beam.runners.worker import log_handler from apache_beam.runners.worker import statesampler -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor _LOGGER = logging.getLogger(__name__) @@ -52,7 +52,7 @@ def Logging(self, request_iterator, context): class FnApiLogRecordHandlerTest(unittest.TestCase): def setUp(self): self.test_logging_service = BeamFnLoggingServicer() - self.server = grpc.server(UnboundedThreadPoolExecutor()) + self.server = grpc.server(thread_pool_executor.shared_unbounded_instance()) beam_fn_api_pb2_grpc.add_BeamFnLoggingServicer_to_server( self.test_logging_service, self.server) self.test_port = self.server.add_insecure_port('[::]:0') diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index eed471d0bc7e..a58dd2facf21 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -64,7 +64,7 @@ from apache_beam.runners.worker.statecache import StateCache from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor from apache_beam.runners.worker.worker_status import FnApiWorkerStatusHandler -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor if TYPE_CHECKING: from apache_beam.portability.api import endpoints_pb2 @@ -186,10 +186,11 @@ def __init__(self, else: self._status_handler = None - # TODO(BEAM-8998) use common UnboundedThreadPoolExecutor to process bundle - # progress once dataflow runner's excessive progress polling is removed. + # TODO(BEAM-8998) use common + # thread_pool_executor.shared_unbounded_instance() to process bundle + # progress once dataflow runner's excessive progress polling is removed. self._report_progress_executor = futures.ThreadPoolExecutor(max_workers=1) - self._worker_thread_pool = UnboundedThreadPoolExecutor() + self._worker_thread_pool = thread_pool_executor.shared_unbounded_instance() self._responses = queue.Queue( ) # type: queue.Queue[beam_fn_api_pb2.InstructionResponse] _LOGGER.info('Initializing SDKHarness with unbounded number of workers.') diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py index a7c3bc394674..d9b808354f20 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py @@ -38,7 +38,7 @@ from apache_beam.portability.api import metrics_pb2 from apache_beam.runners.worker import sdk_worker from apache_beam.runners.worker import statecache -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor _LOGGER = logging.getLogger(__name__) @@ -102,7 +102,7 @@ def _check_fn_registration_multi_request(self, *args): test_controller = BeamFnControlServicer(requests) - server = grpc.server(UnboundedThreadPoolExecutor()) + server = grpc.server(thread_pool_executor.shared_unbounded_instance()) beam_fn_api_pb2_grpc.add_BeamFnControlServicer_to_server( test_controller, server) test_port = server.add_insecure_port("[::]:0") diff --git a/sdks/python/apache_beam/runners/worker/worker_pool_main.py b/sdks/python/apache_beam/runners/worker/worker_pool_main.py index b18961a391bd..f4a4728c373d 100644 --- a/sdks/python/apache_beam/runners/worker/worker_pool_main.py +++ b/sdks/python/apache_beam/runners/worker/worker_pool_main.py @@ -48,7 +48,7 @@ from apache_beam.portability.api import beam_fn_api_pb2 from apache_beam.portability.api import beam_fn_api_pb2_grpc from apache_beam.runners.worker import sdk_worker -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor _LOGGER = logging.getLogger(__name__) @@ -78,7 +78,8 @@ def start( container_executable=None # type: Optional[str] ): # type: (...) -> Tuple[str, grpc.Server] - worker_server = grpc.server(UnboundedThreadPoolExecutor()) + worker_server = grpc.server( + thread_pool_executor.shared_unbounded_instance()) worker_address = 'localhost:%s' % worker_server.add_insecure_port( '[::]:%s' % port) worker_pool = cls( diff --git a/sdks/python/apache_beam/runners/worker/worker_status_test.py b/sdks/python/apache_beam/runners/worker/worker_status_test.py index 0872de4787b0..fea0d7371288 100644 --- a/sdks/python/apache_beam/runners/worker/worker_status_test.py +++ b/sdks/python/apache_beam/runners/worker/worker_status_test.py @@ -28,7 +28,7 @@ from apache_beam.portability.api import beam_fn_api_pb2_grpc from apache_beam.runners.worker.worker_status import FnApiWorkerStatusHandler from apache_beam.testing.util import timeout -from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor +from apache_beam.utils import thread_pool_executor class BeamFnStatusServicer(beam_fn_api_pb2_grpc.BeamFnWorkerStatusServicer): @@ -52,7 +52,7 @@ class FnApiWorkerStatusHandlerTest(unittest.TestCase): def setUp(self): self.num_request = 3 self.test_status_service = BeamFnStatusServicer(self.num_request) - self.server = grpc.server(UnboundedThreadPoolExecutor()) + self.server = grpc.server(thread_pool_executor.shared_unbounded_instance()) beam_fn_api_pb2_grpc.add_BeamFnWorkerStatusServicer_to_server( self.test_status_service, self.server) self.test_port = self.server.add_insecure_port('[::]:0') diff --git a/sdks/python/apache_beam/utils/thread_pool_executor.py b/sdks/python/apache_beam/utils/thread_pool_executor.py index a5443ed5c4b2..c111b9aeff36 100644 --- a/sdks/python/apache_beam/utils/thread_pool_executor.py +++ b/sdks/python/apache_beam/utils/thread_pool_executor.py @@ -134,3 +134,16 @@ def shutdown(self, wait=True): if wait: for worker in self._workers: worker.join() + + +class _SharedUnboundedThreadPoolExecutor(UnboundedThreadPoolExecutor): + def shutdown(self, wait=True): + # Prevent shutting down the shared thread pool + pass + + +_SHARED_UNBOUNDED_THREAD_POOL_EXECUTOR = _SharedUnboundedThreadPoolExecutor() + + +def shared_unbounded_instance(): + return _SHARED_UNBOUNDED_THREAD_POOL_EXECUTOR diff --git a/sdks/python/apache_beam/utils/thread_pool_executor_test.py b/sdks/python/apache_beam/utils/thread_pool_executor_test.py index d9bbae41440b..b9251cad00a1 100644 --- a/sdks/python/apache_beam/utils/thread_pool_executor_test.py +++ b/sdks/python/apache_beam/utils/thread_pool_executor_test.py @@ -30,6 +30,7 @@ # patches unittest.TestCase to be python3 compatible import future.tests.base # pylint: disable=unused-import +from apache_beam.utils import thread_pool_executor from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor @@ -111,6 +112,20 @@ def test_map(self): with self._lock: self.assertEqual(5, len(self._worker_idents)) + def test_shared_shutdown_does_nothing(self): + thread_pool_executor.shared_unbounded_instance().shutdown() + + futures = [] + with thread_pool_executor.shared_unbounded_instance() as executor: + for _ in range(0, 5): + futures.append(executor.submit(self.append_and_sleep, 0.01)) + + for future in futures: + future.result(timeout=10) + + with self._lock: + self.assertEqual(5, len(self._worker_idents)) + if __name__ == '__main__': unittest.main() From 54dccb0143035158685a31920e268f8ff87033e3 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Mon, 1 Jun 2020 17:06:42 -0700 Subject: [PATCH 136/151] wip --- .../sdk/transforms/SerializableFunction.java | 3 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 108 ++++++++++++------ .../gcp/healthcare/HealthcareApiClient.java | 5 +- .../healthcare/HttpHealthcareApiClient.java | 17 ++- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 69 ++++++++++- .../sdk/io/gcp/healthcare/FhirIOWriteIT.java | 51 ++++----- 6 files changed, 183 insertions(+), 70 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java index 3dbe893eef4e..cbd7d01d80fe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.transforms; import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.IOException; import java.io.Serializable; /** @@ -36,5 +37,5 @@ public interface SerializableFunction extends ProcessFunction, Serializable { /** Returns the result of invoking this function on the given input. */ @Override - OutputT apply(InputT input) throws JsonProcessingException; + OutputT apply(InputT input) throws IOException; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 8b6cae2cc336..19ece0754b9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -1419,7 +1419,8 @@ public void initClient() throws IOException { } @ProcessElement - public void create(ProcessContext context) { + public void create(ProcessContext context) + throws IOException { T input = context.element(); String body = formatBodyFunction.apply(input); String type = typeFunction.apply(input); @@ -1447,7 +1448,7 @@ public void create(ProcessContext context) { * @param fhirStore the fhir store * @return the update resources */ - public static UpdateResources update(ValueProvider fhirStore){ + public static UpdateResources update(ValueProvider fhirStore){ return new UpdateResources(fhirStore); } @@ -1458,19 +1459,28 @@ public static UpdateResources update(ValueProvider fhirStore){ * @param fhirStore the fhir store * @return the update resources */ - public static UpdateResources update(String fhirStore){ + public static UpdateResources update(String fhirStore){ return new UpdateResources(fhirStore); } /** * {@link PTransform} for Updating FHIR resources resources. * + * This transform assumes the input {@link PCollection} contains + * {@link KV} of resource name, value pairs and by default will call {@code .toString} to extract + * string values. + * However, the user can override this behavior by specifying a {@link SerializableFunction} with + * custom logic to extract the resource name and body from the {@link KV} in + * {@link UpdateResources#withResourceNameFunction(SerializableFunction)} and + * {@link UpdateResources#withFormatBodyFunction(SerializableFunction)} + * *

    https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/update */ - public static class UpdateResources extends PTransform, Write.Result> { - private final String fhirStore; + public static class UpdateResources extends PTransform, Write.Result> { + private final ValueProvider fhirStore; private SerializableFunction formatBodyFunction; private SerializableFunction resourceNameFunction; + private SerializableFunction etagFunction; public static final TupleTag UPDATE_RESULTS = new TupleTag() {}; private static final Logger LOG = LoggerFactory.getLogger(UpdateResources.class); @@ -1480,9 +1490,17 @@ public static class UpdateResources extends PTransform, Write. * @param fhirStore the fhir store */ UpdateResources(ValueProvider fhirStore) { - this.fhirStore = fhirStore.get(); + this.fhirStore = fhirStore; } + /** + * Instantiates a new Update resources. + * + * @param fhirStore the fhir store + */ + UpdateResources(String fhirStore) { + this.fhirStore = StaticValueProvider.of(fhirStore); + } /** * Add a {@link SerializableFunction} to extract a resource name from the input element. * @@ -1506,16 +1524,19 @@ public UpdateResources withFormatBodyFunction( this.formatBodyFunction = formatBodyFunction; return this; } - /** - * Instantiates a new Update resources. + * With ETag function update resources. * - * @param fhirStore the fhir store + * @param etagFunction ETag function + * @return the update resources */ - UpdateResources(String fhirStore) { - this.fhirStore = fhirStore; + public UpdateResources withETagFunction( + SerializableFunction etagFunction) { + this.etagFunction = etagFunction; + return this; } + @Override public FhirIO.Write.Result expand(PCollection input) { checkArgument( @@ -1527,7 +1548,7 @@ public FhirIO.Write.Result expand(PCollection input) { return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new UpdateFn(fhirStore, formatBodyFunction, resourceNameFunction))) + .apply(ParDo.of(new UpdateFn(fhirStore, formatBodyFunction, resourceNameFunction, etagFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1536,17 +1557,20 @@ static class UpdateFn extends DoFn> { private Counter failedUpdates = Metrics.counter(UpdateFn.class, "failed-updates"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); - private final String fhirStore; + private final ValueProvider fhirStore; private SerializableFunction formatBodyFunction; private SerializableFunction resourceNameFunction; + private SerializableFunction etagFunction; UpdateFn( - String fhirStore, + ValueProvider fhirStore, SerializableFunction formatBodyFunction, - SerializableFunction resourceNameFunction) { + SerializableFunction resourceNameFunction, + @Nullable SerializableFunction etagFunction) { this.fhirStore = fhirStore; this.formatBodyFunction = formatBodyFunction; this.resourceNameFunction = resourceNameFunction; + this.etagFunction = etagFunction; } @Setup @@ -1555,14 +1579,19 @@ public void initClient() throws IOException { } @ProcessElement - public void update(ProcessContext context) { + public void update(ProcessContext context) + throws IOException { T input = context.element(); String body = formatBodyFunction.apply(input); try { // Validate that data was set to valid JSON. mapper.readTree(body); String resourceName = resourceNameFunction.apply(input); - HttpBody result = client.fhirUpdate(fhirStore, resourceName, body); + String etag = null; + if (etagFunction != null){ + etag = etagFunction.apply(input); + } + HttpBody result = client.fhirUpdate(fhirStore.get(), resourceName, body, etag); context.output(UPDATE_RESULTS, result.getData()); } catch (IOException | HealthcareHttpException e) { failedUpdates.inc(); @@ -1600,10 +1629,11 @@ public static ConditionalUpdate conditionalUpdate(String fhirStore){ *

    https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/conditionalUpdate */ public static class ConditionalUpdate extends PTransform, Write.Result> { - private final String fhirStore; + private final ValueProvider fhirStore; private SerializableFunction> searchParametersFunction; private SerializableFunction typeFunction; private SerializableFunction formatBodyFunction; + private SerializableFunction etagFunction; /** * Instantiates a new Conditional update. @@ -1611,9 +1641,19 @@ public static class ConditionalUpdate extends PTransform, Writ * @param fhirStore the fhir store */ ConditionalUpdate(ValueProvider fhirStore) { - this.fhirStore = fhirStore.get(); + this.fhirStore = fhirStore; + } + + /** + * Instantiates a new Conditional update. + * + * @param fhirStore the fhir store + */ + ConditionalUpdate(String fhirStore) { + this.fhirStore = StaticValueProvider.of(fhirStore); } + /** * With search parameters function conditional update. * @@ -1649,13 +1689,9 @@ public ConditionalUpdate withFormatBodyFunction( return this; } - /** - * Instantiates a new Conditional update. - * - * @param fhirStore the fhir store - */ - ConditionalUpdate(String fhirStore) { - this.fhirStore = fhirStore; + public ConditionalUpdate withEtagFunction(SerializableFunction etagFunction) { + this.etagFunction = etagFunction; + return this; } @Override @@ -1678,7 +1714,8 @@ public FhirIO.Write.Result expand(PCollection input) { .apply( ParDo.of( new ConditionalUpdateFn( - fhirStore, typeFunction, searchParametersFunction, formatBodyFunction))) + fhirStore, typeFunction, searchParametersFunction, formatBodyFunction, + etagFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1688,20 +1725,23 @@ static class ConditionalUpdateFn extends DoFn> { Metrics.counter(ConditionalUpdateFn.class, "failed-conditional-updates"); private transient HealthcareApiClient client; private final ObjectMapper mapper = new ObjectMapper(); - private final String fhirStore; + private final ValueProvider fhirStore; private SerializableFunction> searchParametersFunction; private SerializableFunction typeFunction; private SerializableFunction formatBodyFunction; + private SerializableFunction etagFunction; ConditionalUpdateFn( - String fhirStore, + ValueProvider fhirStore, SerializableFunction typeFunction, SerializableFunction> searchParametersFunction, - SerializableFunction formatBodyFunction) { + SerializableFunction formatBodyFunction, + @Nullable SerializableFunction etagFunction) { this.fhirStore = fhirStore; this.typeFunction = typeFunction; this.searchParametersFunction = searchParametersFunction; this.formatBodyFunction = formatBodyFunction; + this.etagFunction = etagFunction; } @Setup @@ -1710,7 +1750,7 @@ public void initClient() throws IOException { } @ProcessElement - public void conditionalUpdate(ProcessContext context) { + public void conditionalUpdate(ProcessContext context) throws IOException { T input = context.element(); String type = typeFunction.apply(input); String body = formatBodyFunction.apply(input); @@ -1718,7 +1758,11 @@ public void conditionalUpdate(ProcessContext context) { // Validate that data was set to valid JSON. mapper.readTree(body); Map searchParameters = searchParametersFunction.apply(input); - client.fhirConditionalUpdate(fhirStore, type, body, searchParameters); + String etag = null; + if (etagFunction != null){ + etag = etagFunction.apply(input); + } + client.fhirConditionalUpdate(fhirStore.get(), type, body, searchParameters, etag); } catch (IOException | HealthcareHttpException e) { failedConditionalUpdates.inc(); context.output(HealthcareIOError.of(body, e)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index 8083356bfe0e..3f553f1c7ec7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -157,10 +157,11 @@ HttpBody fhirConditionalUpdate( String fhirStore, String relativeResourceName, String resource, - Map searchParameters) + Map searchParameters, + @Nullable String etag) throws IOException, HealthcareHttpException; - HttpBody fhirUpdate(String fhirStore, String type, String resource) + HttpBody fhirUpdate(String fhirStore, String type, String resource, @Nullable String etag) throws IOException, HealthcareHttpException; /** diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 78c7bcbb3fcd..dbad6125dbc1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -86,6 +86,7 @@ public class HttpHealthcareApiClient implements HealthcareApiClient, Serializabl private static final String FHIRSTORE_HEADER_CONTENT_TYPE = "application/fhir+json"; private static final String FHIRSTORE_HEADER_ACCEPT = "application/fhir+json; charset=utf-8"; private static final String FHIRSTORE_HEADER_ACCEPT_CHARSET = "utf-8"; + private static final String IF_MATCH_HEADER = "If-Match"; private static final Logger LOG = LoggerFactory.getLogger(HttpHealthcareApiClient.class); private transient CloudHealthcare client; public transient HttpClient httpClient; @@ -531,20 +532,32 @@ public HttpBody fhirCreate( @Override public HttpBody fhirConditionalUpdate( - String fhirStore, String type, String resource, Map searchParameters) + String fhirStore, String type, String resource, Map searchParameters, + @Nullable String etag) throws IOException, HealthcareHttpException { + Map headers = new HashMap<>(); + if (etag != null) { + headers.put(IF_MATCH_HEADER, etag); + } return executeFhirHttpRequest( FhirHttpRequest.of(fhirStore, resource) + .setHeaders(headers) .setPathSuffix("/" + type) .setParameters(searchParameters) .setMethod(Method.PUT)); } @Override - public HttpBody fhirUpdate(String fhirStore, String relativeResourceName, String resource) + public HttpBody fhirUpdate(String fhirStore, String relativeResourceName, String resource, + @Nullable String etag) throws IOException, HealthcareHttpException { + Map headers = new HashMap<>(); + if (etag != null) { + headers.put(IF_MATCH_HEADER, etag); + } return executeFhirHttpRequest( FhirHttpRequest.of(fhirStore, resource) + .setHeaders(headers) .setPathSuffix("/" + relativeResourceName) .setMethod(Method.PUT)); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 100c26efc0be..61af5fa8f50b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.healthcare; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.http.HttpHeaders; import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.http.javanet.NetHttpTransport; @@ -39,16 +40,52 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; +import org.apache.beam.sdk.transforms.SerializableFunction; class FhirIOTestUtil { public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests"; + + public static class ExtractIDSearchQuery implements + SerializableFunction { + private ObjectMapper mapper; + + ExtractIDSearchQuery(){ + mapper = new ObjectMapper(); + } + + @Override + public String apply(String resource) throws IOException { + Map map = mapper.readValue(resource.toString().getBytes(), Map.class); + String id = map.get("id"); + return String.format("_id=%s", id); + } + } + + public static class GetByKey implements + SerializableFunction { + private final String key; + private ObjectMapper mapper; + + public GetByKey(String key){ + this.key = key; + mapper = new ObjectMapper(); + } + + @Override + public String apply(String resource) throws IOException { + Map map = mapper.readValue(resource.getBytes(), Map.class); + return map.get(key); + } + } + // TODO read initial resources function. // TODO read update resources function. // TODO spot check resource update utility. - private static Stream readPrettyBundles(String version) { - ClassLoader classLoader = FhirIOTestUtil.class.getClassLoader(); - Path resourceDir = Paths.get("build", "resources", "test", version); + + private static Stream readAllTestResources(String subDir, String version ) { + Path resourceDir = Paths.get("build", "resources", "test", + subDir, version); String absolutePath = resourceDir.toFile().getAbsolutePath(); File dir = new File(absolutePath); File[] fhirJsons = dir.listFiles(); @@ -63,8 +100,15 @@ private static Stream readPrettyBundles(String version) { } }) .map(String::new); + + } + private static Stream readPrettyBundles(String version) { + return readAllTestResources("transactional_bundles", version); } + private static Stream readPrettyResources(String version) { + return readAllTestResources("resources", version); + } // Could generate more messages at scale using a tool like // https://synthetichealth.github.io/synthea/ if necessary chose not to avoid the dependency. static final List DSTU2_PRETTY_BUNDLES = @@ -84,6 +128,25 @@ private static Stream readPrettyBundles(String version) { BUNDLES = Collections.unmodifiableMap(m); } + // Could generate more messages at scale using a tool like + // https://synthetichealth.github.io/synthea/ if necessary chose not to avoid the dependency. + static final List DSTU2_PRETTY_RESOURCES = + readPrettyResources("DSTU2").collect(Collectors.toList()); + static final List STU3_PRETTY_RESOURCES = + readPrettyResources("STU3").collect(Collectors.toList()); + static final List R4_PRETTY_RESOURCES= + readPrettyResources("R4").collect(Collectors.toList()); + + static final Map> RESOURCES; + + static { + Map> m = new HashMap<>(); + m.put("DSTU2", DSTU2_PRETTY_RESOURCES); + m.put("STU3", STU3_PRETTY_RESOURCES); + m.put("R4", R4_PRETTY_RESOURCES); + RESOURCES = Collections.unmodifiableMap(m); + } + /** Populate the test resources into the FHIR store and returns a list of resource IDs. */ static void executeFhirBundles(HealthcareApiClient client, String fhirStore, List bundles) throws IOException, HealthcareHttpException { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java index 8d46186c472b..4f127c050add 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java @@ -19,24 +19,27 @@ import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.BUNDLES; import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.RESOURCES; import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; import java.io.IOException; import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.KV; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -139,24 +142,13 @@ public void testFhirIO_CreateResources() { (Result) pipeline .apply(Create.of(BUNDLES.get(version))) .apply(FhirIO.createResources(options.getFhirStore()) - .withTypeFunction((String resource) -> { - ObjectMapper mapper = new ObjectMapper(); - Map map = mapper.readValue(resource, Map.class); - return map.get("resourceType"); - - }) - .withIfNotExistFunction((String resource) -> { - ObjectMapper mapper = new ObjectMapper(); - Map map = mapper.readValue(resource, Map.class); - String id = map.get("id"); - return String.format("_id=%s", id); - }) - .withFormatBodyFunction((String x) -> x) + .withTypeFunction(new GetByKey("resourceType")) + .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) + .withFormatBodyFunction(x -> x) ); - // TODO have a known number of pre-condition failed case and assert it appears in error queue - PAssert.thatSingleton(writeResult.getFailedBodies().apply(Count.globally())).isEqualTo(1L); + PAssert.that(writeResult.getFailedBodies()).empty(); pipeline.run().waitUntilFinish(); } @@ -164,14 +156,14 @@ public void testFhirIO_CreateResources() { @Test public void testFhirIO_Update() { // TODO write initial resources to FHIR - // use beam to perform updates FhirIO.Write.Result writeResult = (Result) pipeline - .apply(Create.of(BUNDLES.get(version))) - .apply(FhirIO.update(options.getFhirStore()) - .withFormatBodyFunction(x -> "{}") - .withResourceNameFunction(x -> "name") - ); + .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) + .apply("Update Resources", + FhirIO.>update(options.getFhirStore()) + .withResourceNameFunction(x -> x.getKey()) + .withFormatBodyFunction(x -> ((KV) x).getValue())); PAssert.that(writeResult.getFailedBodies()).empty(); @@ -184,13 +176,12 @@ public void testFhirIO_ConditionalUpdate() { // TODO write initial resources to FHIR FhirIO.Write.Result writeResult = (Result) pipeline - .apply(Create.of(BUNDLES.get(version))) - .apply(FhirIO.conditionalUpdate(options.getFhirStore()) - .withTypeFunction(x -> "patient") - .withFormatBodyFunction(x -> "{}") - .withSearchParametersFunction(x-> new HashMap()) - ); + .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply("Conditional Update Resources", + FhirIO.conditionalUpdate(options.getFhirStore()) + .withTypeFunction(x -> "patient") + .withFormatBodyFunction(x -> "{}") + .withSearchParametersFunction(x -> new HashMap<>())); // TODO spot check update results - // TODO have a known number of pre-condition failed case and assert it appears in error queue } } From 01c11e7211937bde3c238fe3639f9dfe7774d093 Mon Sep 17 00:00:00 2001 From: darshanj Date: Tue, 2 Jun 2020 11:54:54 +0800 Subject: [PATCH 137/151] [BEAM-9825] | Implement Intersect,Union,Except transforms (#11610) * [BEAM-9825] | Implement Intersect,Union,Except transforms --- .../org/apache/beam/sdk/transforms/Sets.java | 680 ++++++++++++++++++ .../apache/beam/sdk/transforms/SetsTest.java | 324 +++++++++ .../sql/impl/rel/BeamSetOperatorRelBase.java | 43 +- 3 files changed, 1027 insertions(+), 20 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sets.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sets.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sets.java new file mode 100644 index 000000000000..16a728167b24 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sets.java @@ -0,0 +1,680 @@ +/* + * 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.beam.sdk.transforms; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; + +/** + * The {@code PTransform}s that allow to compute different set functions across {@link + * PCollection}s. + * + *

    They come in two variants. 1. Between two {@link PCollection} 2. Between two or more {@link + * PCollection} in a {@link PCollectionList}. + * + *

    Following {@code PTransform}s follows SET DISTINCT semantics: intersectDistinct, + * expectDistinct, unionDistinct + * + *

    Following {@code PTransform}s follows SET ALL semantics: intersectAll, expectAll, unionAll + * + *

    For example, the following demonstrates intersectDistinct between two collections {@link + * PCollection}s. + * + *

    {@code
    + * Pipeline p = ...;
    + *
    + * PCollection left = p.apply(Create.of("1", "2", "3", "3", "4", "5"));
    + * PCollection right = p.apply(Create.of("1", "3", "4", "4", "6"));
    + *
    + * PCollection results =
    + *     left.apply(SetFns.intersectDistinct(right)); // results will be PCollection containing: "1","3","4"
    + *
    + * }
    + * + *

    For example, the following demonstrates intersectDistinct between three collections {@link + * PCollection}s in a {@link PCollectionList}. + * + *

    {@code
    + * Pipeline p = ...;
    + *
    + * PCollection first = p.apply(Create.of("1", "2", "3", "3", "4", "5"));
    + * PCollection second = p.apply(Create.of("1", "3", "4", "4", "6"));
    + * PCollection third = p.apply(Create.of("3", "4", "4"));
    + *
    + * // Following example will perform (first intersect second) intersect third.
    + * PCollection results =
    + *     PCollectionList.of(first).and(second).and(third)
    + *     .apply(SetFns.intersectDistinct()); // results will be PCollection containing: "3","4"
    + *
    + * }
    + */ +public class Sets { + + /** + * Returns a new {@code PTransform} transform that follows SET DISTINCT semantics to compute the + * intersection with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} will all distinct elements that present in + * both pipeline is constructed and provided {@link PCollection}. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}). Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "2", "3", "3", "4", "5"));
    +   * PCollection right = p.apply(Create.of("1", "3", "4", "4", "6"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.intersectDistinct(right)); // results will be PCollection containing: "1","3","4"
    +   *
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> intersectDistinct( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, intersectDistinct()); + } + + /** + * Returns a {@code PTransform} that takes a {@code PCollectionList>} and returns a + * {@code PCollection} containing the intersection of collections done in order for all + * collections in {@code PCollectionList}. + * + *

    Returns a new {@code PTransform} transform that follows SET DISTINCT semantics which takes a + * {@code PCollectionList>} and returns a {@code PCollection} containing the + * intersection of collections done in order for all collections in {@code PCollectionList}. + * + *

    The elements of the output {@link PCollection} will have all distinct elements that are + * present in both pipeline is constructed and next {@link PCollection} in the list and applied to + * all collections in order. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection first = p.apply(Create.of("1", "2", "3", "3", "4", "5"));
    +   * PCollection second = p.apply(Create.of("1", "3", "4", "4", "6"));
    +   * PCollection third = p.apply(Create.of("3", "4", "4"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.intersectDistinct()); // results will be PCollection containing: "3","4"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static PTransform, PCollection> intersectDistinct() { + SerializableBiFunction intersectFn = + (numberOfElementsinLeft, numberOfElementsinRight) -> + (numberOfElementsinLeft > 0 && numberOfElementsinRight > 0) ? 1L : 0L; + return new SetImplCollections<>(intersectFn); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics to compute the + * intersection with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} which will follow INTESECT_ALL Semantics as + * follows: Given there are m elements on pipeline which is constructed {@link PCollection} (left) + * and n elements on in provided {@link PCollection} (right): - it will output MIN(m - n, 0) + * elements of left for all elements which are present in both left and right. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "1", "1", "2", "3", "3", "4", "5"));
    +   * PCollection right = p.apply(Create.of("1", "1", "3", "4", "4", "6"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.intersectAll(right)); // results will be PCollection containing: "1","1","3","4"
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> intersectAll( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, intersectAll()); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics which takes a {@code + * PCollectionList>} and returns a {@code PCollection} containing the + * intersection all of collections done in order for all collections in {@code + * PCollectionList}. + * + *

    The elements of the output {@link PCollection} which will follow INTERSECT_ALL semantics. + * Output is calculated as follows: Given there are m elements on pipeline which is constructed + * {@link PCollection} (left) and n elements on in provided {@link PCollection} (right): - it will + * output MIN(m - n, 0) elements of left for all elements which are present in both left and + * right. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   * PCollection first = p.apply(Create.of("1", "1", "1", "2", "3", "3", "4", "5"));
    +   * PCollection second = p.apply(Create.of("1", "1", "3", "4", "4", "6"));
    +   * PCollection third = p.apply(Create.of("1", "5"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.intersectAll()); // results will be PCollection containing: "1"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static PTransform, PCollection> intersectAll() { + return new SetImplCollections<>(Math::min); + } + + /** + * Returns a new {@code PTransform} transform that follows SET DISTINCT semantics to compute the + * difference (except) with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} will all distinct elements that present in + * pipeline is constructed but not present in provided {@link PCollection}. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "1", "1", "2", "3", "3","4", "5"));
    +   * PCollection right = p.apply(Create.of("1", "1", "3", "4", "4", "6"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.exceptDistinct(right)); // results will be PCollection containing: "2","5"
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> exceptDistinct( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, exceptDistinct()); + } + + /** + * Returns a {@code PTransform} that takes a {@code PCollectionList>} and returns a + * {@code PCollection} containing the difference (except) of collections done in order for all + * collections in {@code PCollectionList}. + * + *

    Returns a new {@code PTransform} transform that follows SET DISTINCT semantics which takes a + * {@code PCollectionList>} and returns a {@code PCollection} containing the + * difference (except) of collections done in order for all collections in {@code + * PCollectionList}. + * + *

    The elements of the output {@link PCollection} will have all distinct elements that are + * present in pipeline is constructed but not present in next {@link PCollection} in the list and + * applied to all collections in order. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   * PCollection first = p.apply(Create.of("1", "1", "1", "2", "3", "3", "4", "5"));
    +   * PCollection second = p.apply(Create.of("1", "1", "3", "4", "4", "6"));
    +   *
    +   * PCollection third = p.apply(Create.of("1", "2", "2"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.exceptDistinct()); // results will be PCollection containing: "5"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static PTransform, PCollection> exceptDistinct() { + SerializableBiFunction exceptFn = + (numberOfElementsinLeft, numberOfElementsinRight) -> + numberOfElementsinLeft > 0 && numberOfElementsinRight == 0 ? 1L : 0L; + return new SetImplCollections<>(exceptFn); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics to compute the + * difference all (exceptAll) with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} which will follow EXCEPT_ALL Semantics as + * follows: Given there are m elements on pipeline which is constructed {@link PCollection} (left) + * and n elements on in provided {@link PCollection} (right): - it will output m elements of left + * for all elements which are present in left but not in right. - it will output MAX(m - n, 0) + * elements of left for all elements which are present in both left and right. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "1", "1", "2", "3", "3", "3", "4", "5"));
    +   * PCollection right = p.apply(Create.of("1", "3", "4", "4", "6"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.exceptAll(right)); // results will be PCollection containing: "1","1","2","3","3","5"
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> exceptAll( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, exceptAll()); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics which takes a {@code + * PCollectionList>} and returns a {@code PCollection} containing the difference + * all (exceptAll) of collections done in order for all collections in {@code PCollectionList}. + * + *

    The elements of the output {@link PCollection} which will follow EXCEPT_ALL semantics. + * Output is calculated as follows: Given there are m elements on pipeline which is constructed + * {@link PCollection} (left) and n elements on in provided {@link PCollection} (right): - it will + * output m elements of left for all elements which are present in left but not in right. - it + * will output MAX(m - n, 0) elements of left for all elements which are present in both left and + * right. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   * PCollection first = p.apply(Create.of("1", "1", "1", "2", "3", "3", "3", "4", "5"));
    +   * PCollection second = p.apply(Create.of("1", "3", "4", "4", "6"));
    +   * PCollection third = p.apply(Create.of("1", "5"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.exceptAll()); // results will be PCollection containing: "1","2","3","3"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static PTransform, PCollection> exceptAll() { + SerializableBiFunction exceptFn = + (numberOfElementsinLeft, numberOfElementsinRight) -> + Math.max(numberOfElementsinLeft - numberOfElementsinRight, 0L); + return new SetImplCollections<>(exceptFn); + } + + /** + * Returns a new {@code PTransform} transform that follows SET DISTINCT semantics to compute the + * union with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} will all distinct elements that present in + * pipeline is constructed or present in provided {@link PCollection}. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "1", "2"));
    +   * PCollection right = p.apply(Create.of("1", "3", "4", "4"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.unionDistinct(right)); // results will be PCollection containing: "1","2","3","4"
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> unionDistinct( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, unionDistinct()); + } + + /** + * Returns a new {@code PTransform} transform that follows SET DISTINCT semantics which takes a + * {@code PCollectionList>} and returns a {@code PCollection} containing the + * union of collections done in order for all collections in {@code PCollectionList}. + * + *

    The elements of the output {@link PCollection} will have all distinct elements that are + * present in pipeline is constructed or present in next {@link PCollection} in the list and + * applied to all collections in order. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   * PCollection first = p.apply(Create.of("1", "1", "2"));
    +   * PCollection second = p.apply(Create.of("1", "3", "4", "4"));
    +   *
    +   * PCollection third = p.apply(Create.of("1", "5"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.unionDistinct()); // results will be PCollection containing: "1","2","3","4","5"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static PTransform, PCollection> unionDistinct() { + SerializableBiFunction unionFn = + (numberOfElementsinLeft, numberOfElementsinRight) -> 1L; + return new SetImplCollections<>(unionFn); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics to compute the + * unionAll with provided {@code PCollection}. + * + *

    The argument should not be modified after this is called. + * + *

    The elements of the output {@link PCollection} which will follow UNION_ALL semantics as + * follows: Given there are m elements on pipeline which is constructed {@link PCollection} (left) + * and n elements on in provided {@link PCollection} (right): - it will output m elements of left + * and m elements of right. + * + *

    Note that this transform requires that the {@code Coder} of the all {@code PCollection} + * to be deterministic (see {@link Coder#verifyDeterministic()}). If the collection {@code Coder} + * is not deterministic, an exception is thrown at pipeline construction time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the input {@code PCollection} + * + *

    {@code
    +   * Pipeline p = ...;
    +   *
    +   * PCollection left = p.apply(Create.of("1", "1", "2"));
    +   * PCollection right = p.apply(Create.of("1", "3", "4", "4"));
    +   *
    +   * PCollection results =
    +   *     left.apply(SetFns.unionAll(right)); // results will be PCollection containing: "1","1","1","2","3","4","4"
    +   * }
    + * + * @param the type of the elements in the input and output {@code PCollection}s. + */ + public static PTransform, PCollection> unionAll( + PCollection rightCollection) { + checkNotNull(rightCollection, "rightCollection argument is null"); + return new SetImpl<>(rightCollection, unionAll()); + } + + /** + * Returns a new {@code PTransform} transform that follows SET ALL semantics which takes a {@code + * PCollectionList>} and returns a {@code PCollection} containing the unionAll + * of collections done in order for all collections in {@code PCollectionList}. + * + *

    The elements of the output {@link PCollection} which will follow UNION_ALL semantics. Output + * is calculated as follows: Given there are m elements on pipeline which is constructed {@link + * PCollection} (left) and n elements on in provided {@link PCollection} (right): - it will output + * m elements of left and m elements of right. + * + *

    Note that this transform requires that the {@code Coder} of the all inputs {@code + * PCollection} to be deterministic (see {@link Coder#verifyDeterministic()}). If the + * collection {@code Coder} is not deterministic, an exception is thrown at pipeline construction + * time. + * + *

    All inputs must have equal {@link WindowFn}s and compatible triggers (see {@link + * Trigger#isCompatible(Trigger)}).Triggers with multiple firings may lead to nondeterministic + * results since the this {@code PTransform} is only computed over each individual firing. + * + *

    By default, the output {@code PCollection} encodes its elements using the same {@code + * Coder} as that of the first {@code PCollection} in {@code PCollectionList}. + * + *

    {@code
    +   * Pipeline p = ...;
    +   * PCollection first = p.apply(Create.of("1", "1", "2"));
    +   * PCollection second = p.apply(Create.of("1", "3", "4", "4"));
    +   * PCollection third = p.apply(Create.of("1", "5"));
    +   *
    +   * // Following example will perform (first intersect second) intersect third.
    +   * PCollection results =
    +   *     PCollectionList.of(first).and(second).and(third)
    +   *     .apply(SetFns.unionAll()); // results will be PCollection containing: "1","1","1","1","2","3","4","4","5"
    +   *
    +   * }
    + * + * @param the type of the elements in the input {@code PCollectionList} and output {@code + * PCollection}s. + */ + public static Flatten.PCollections unionAll() { + return Flatten.pCollections(); + } + + private static class SetImpl extends PTransform, PCollection> { + + private final transient PCollection rightCollection; + private final PTransform, PCollection> listTransformFn; + + private SetImpl( + PCollection rightCollection, + PTransform, PCollection> listTransformFn) { + this.rightCollection = rightCollection; + this.listTransformFn = listTransformFn; + } + + @Override + public PCollection expand(PCollection leftCollection) { + return PCollectionList.of(leftCollection).and(rightCollection).apply(listTransformFn); + } + } + + private static class SetImplCollections + extends PTransform, PCollection> { + + private final SerializableBiFunction fn; + + private SetImplCollections(SerializableBiFunction fn) { + this.fn = fn; + } + + @Override + public PCollection expand(PCollectionList input) { + List> all = input.getAll(); + MapElements> elementToVoid = + MapElements.via( + new SimpleFunction>() { + @Override + public KV apply(T element) { + return KV.of(element, null); + } + }); + + checkArgument(all.size() > 1, "must have at least two input to a PCollectionList"); + + PCollection first = all.get(0); + Pipeline pipeline = first.getPipeline(); + String firstName = first.getName(); + + List> allTags = new ArrayList<>(); + KeyedPCollectionTuple keyedPCollectionTuple = KeyedPCollectionTuple.empty(pipeline); + + for (PCollection col : all) { + TupleTag tag = new TupleTag<>(); + + PCollection> kvOfElementAndVoid = + col.apply("PrepareKVs" + col.getName(), elementToVoid); + + allTags.add(tag); + keyedPCollectionTuple = keyedPCollectionTuple.and(tag, kvOfElementAndVoid); + } + + PCollection> coGbkResults = + keyedPCollectionTuple.apply("CBKAll" + firstName, CoGroupByKey.create()); + + // TODO: lift combiners through the CoGBK. + PCollection results = + coGbkResults.apply( + "FilterSetElement" + firstName, + ParDo.of( + new DoFn, T>() { + + @ProcessElement + public void processElement(ProcessContext c) { + KV elementGroups = c.element(); + CoGbkResult value = elementGroups.getValue(); + T element = elementGroups.getKey(); + + long numberOfOutputs = Iterables.size(value.getAll(allTags.get(0))); + List> tail = allTags.subList(1, allTags.size()); + + for (TupleTag tag : tail) { + long nextSize = Iterables.size(value.getAll(tag)); + numberOfOutputs = fn.apply(numberOfOutputs, nextSize); + } + for (long i = 0L; i < numberOfOutputs; i++) { + c.output(element); + } + } + })); + + return results.setCoder(first.getCoder()); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java new file mode 100644 index 000000000000..ed07a65b0219 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java @@ -0,0 +1,324 @@ +/* + * 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.beam.sdk.transforms; + +import static junit.framework.TestCase.assertEquals; + +import java.util.Arrays; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SetsTest { + + @Rule public final TestPipeline p = TestPipeline.create(); + + Schema schema = Schema.builder().addStringField("alphabet").build(); + + static PCollection first; + static PCollection second; + static PCollection firstRows; + static PCollection secondRows; + + private Iterable toRows(String... values) { + return Iterables.transform( + Arrays.asList(values), (elem) -> Row.withSchema(schema).addValues(elem).build()); + } + + @Before + public void setup() { + final String[] firstData = {"a", "a", "a", "b", "b", "c", "d", "d", "g", "g", "h", "h"}; + final String[] secondData = {"a", "a", "b", "b", "b", "c", "d", "d", "e", "e", "f", "f"}; + + first = p.apply("first", Create.of(Arrays.asList(firstData))); + second = p.apply("second", Create.of(Arrays.asList(secondData))); + + firstRows = p.apply("firstRows", Create.of(toRows(firstData)).withRowSchema(schema)); + secondRows = p.apply("secondRows", Create.of(toRows(secondData)).withRowSchema(schema)); + } + + @Test + @Category(NeedsRunner.class) + public void testIntersection() { + PAssert.that(first.apply("strings", Sets.intersectDistinct(second))) + .containsInAnyOrder("a", "b", "c", "d"); + + PCollection results = firstRows.apply("rows", Sets.intersectDistinct(secondRows)); + + PAssert.that(results).containsInAnyOrder(toRows("a", "b", "c", "d")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testIntersectionCollectionList() { + + PCollection third = p.apply("third", Create.of(Arrays.asList("b", "b", "c", "f"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("b", "b", "c", "f"))); + + PAssert.that( + PCollectionList.of(first) + .and(second) + .and(third) + .apply("stringsCols", Sets.intersectDistinct())) + .containsInAnyOrder("b", "c"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.intersectDistinct()); + + PAssert.that(results).containsInAnyOrder(toRows("b", "c")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testIntersectionAll() { + + PAssert.that(first.apply("strings", Sets.intersectAll(second))) + .containsInAnyOrder("a", "a", "b", "b", "c", "d", "d"); + + PCollection results = firstRows.apply("rows", Sets.intersectAll(secondRows)); + + PAssert.that(results).containsInAnyOrder(toRows("a", "a", "b", "b", "c", "d", "d")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testIntersectionAllCollectionList() { + PCollection third = p.apply("third", Create.of(Arrays.asList("a", "b", "f"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("a", "b", "f"))); + + PAssert.that( + PCollectionList.of(first) + .and(second) + .and(third) + .apply("stringsCols", Sets.intersectAll())) + .containsInAnyOrder("a", "b"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.intersectAll()); + + PAssert.that(results).containsInAnyOrder(toRows("a", "b")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testExcept() { + + PAssert.that(first.apply("strings", Sets.exceptDistinct(second))).containsInAnyOrder("g", "h"); + + PCollection results = firstRows.apply("rows", Sets.exceptDistinct(secondRows)); + + PAssert.that(results).containsInAnyOrder(toRows("g", "h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testExceptCollectionList() { + PCollection third = p.apply("third", Create.of(Arrays.asList("a", "b", "b", "g", "g"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("a", "b", "b", "g", "g"))); + + PAssert.that( + PCollectionList.of(first) + .and(second) + .and(third) + .apply("stringsCols", Sets.exceptDistinct())) + .containsInAnyOrder("h"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.exceptDistinct()); + + PAssert.that(results).containsInAnyOrder(toRows("h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testExceptAll() { + + PAssert.that(first.apply("strings", Sets.exceptAll(second))) + .containsInAnyOrder("a", "g", "g", "h", "h"); + + PCollection results = firstRows.apply("rows", Sets.exceptAll(secondRows)); + + PAssert.that(results).containsInAnyOrder(toRows("a", "g", "g", "h", "h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testExceptAllCollectionList() { + PCollection third = p.apply("third", Create.of(Arrays.asList("a", "b", "b", "g", "f"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("a", "b", "b", "g"))); + + PAssert.that( + PCollectionList.of(first).and(second).and(third).apply("stringsCols", Sets.exceptAll())) + .containsInAnyOrder("g", "h", "h"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.exceptAll()); + + PAssert.that(results).containsInAnyOrder(toRows("g", "h", "h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testUnion() { + + PAssert.that(first.apply("strings", Sets.unionDistinct(second))) + .containsInAnyOrder("a", "b", "c", "d", "e", "f", "g", "h"); + + PCollection results = firstRows.apply("rows", Sets.unionDistinct(secondRows)); + + PAssert.that(results).containsInAnyOrder(toRows("a", "b", "c", "d", "e", "f", "g", "h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testUnionCollectionList() { + PCollection third = p.apply("third", Create.of(Arrays.asList("a", "k", "k"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("a", "k", "k"))); + + PAssert.that( + PCollectionList.of(first) + .and(second) + .and(third) + .apply("stringsCols", Sets.unionDistinct())) + .containsInAnyOrder("a", "b", "c", "d", "e", "f", "g", "h", "k"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.unionDistinct()); + + PAssert.that(results).containsInAnyOrder(toRows("a", "b", "c", "d", "e", "f", "g", "h", "k")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testUnionAll() { + + PAssert.that(first.apply("strings", Sets.unionAll(second))) + .containsInAnyOrder( + "a", "a", "a", "a", "a", "b", "b", "b", "b", "b", "c", "c", "d", "d", "d", "d", "e", + "e", "f", "f", "g", "g", "h", "h"); + + PCollection results = firstRows.apply("rows", Sets.unionAll(secondRows)); + + PAssert.that(results) + .containsInAnyOrder( + toRows( + "a", "a", "a", "a", "a", "b", "b", "b", "b", "b", "c", "c", "d", "d", "d", "d", "e", + "e", "f", "f", "g", "g", "h", "h")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testUnionAllCollections() { + + PCollection third = p.apply("third", Create.of(Arrays.asList("a", "b", "b", "k", "k"))); + PCollection thirdRows = p.apply("thirdRows", Create.of(toRows("a", "b", "b", "k", "k"))); + + PAssert.that( + PCollectionList.of(first).and(second).and(third).apply("stringsCols", Sets.unionAll())) + .containsInAnyOrder( + "a", "a", "a", "a", "a", "a", "b", "b", "b", "b", "b", "b", "b", "c", "c", "d", "d", + "d", "d", "e", "e", "f", "f", "g", "g", "h", "h", "k", "k"); + + PCollection results = + PCollectionList.of(firstRows) + .and(secondRows) + .and(thirdRows) + .apply("rowCols", Sets.unionAll()); + + PAssert.that(results) + .containsInAnyOrder( + toRows( + "a", "a", "a", "a", "a", "a", "b", "b", "b", "b", "b", "b", "b", "c", "c", "d", "d", + "d", "d", "e", "e", "f", "f", "g", "g", "h", "h", "k", "k")); + + assertEquals(schema, results.getSchema()); + + p.run(); + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java index ab76a7b897a9..bdc5f4c1bffa 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java @@ -20,15 +20,11 @@ import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument; import java.io.Serializable; -import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSetOperatorsTransforms; -import org.apache.beam.sdk.schemas.transforms.CoGroup; -import org.apache.beam.sdk.schemas.transforms.CoGroup.By; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sets; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; /** @@ -75,20 +71,27 @@ public PCollection expand(PCollectionList inputs) { + rightWindow); } - // TODO: We may want to preaggregate the counts first using Group instead of calling CoGroup and - // measuring the - // iterable size. If on average there are duplicates in the input, this will be faster. - final String lhsTag = "lhs"; - final String rhsTag = "rhs"; - PCollection joined = - PCollectionTuple.of(lhsTag, leftRows, rhsTag, rightRows) - .apply("CoGroup", CoGroup.join(By.fieldNames("*"))); - return joined - .apply( - "FilterResults", - ParDo.of( - new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn( - lhsTag, rhsTag, opType, all))) - .setRowSchema(joined.getSchema().getField("key").getType().getRowSchema()); + switch (opType) { + case UNION: + if (all) { + return leftRows.apply(Sets.unionAll(rightRows)); + } else { + return leftRows.apply(Sets.unionDistinct(rightRows)); + } + case INTERSECT: + if (all) { + return leftRows.apply(Sets.intersectAll(rightRows)); + } else { + return leftRows.apply(Sets.intersectDistinct(rightRows)); + } + case MINUS: + if (all) { + return leftRows.apply(Sets.exceptAll(rightRows)); + } else { + return leftRows.apply(Sets.exceptDistinct(rightRows)); + } + default: + throw new IllegalStateException("Unexpected set operation value: " + opType); + } } } From b6ca2aba5a0141eed5bed29a9948e2c65874254f Mon Sep 17 00:00:00 2001 From: Michal Walenia <32354134+mwalenia@users.noreply.github.com> Date: Tue, 2 Jun 2020 08:56:14 +0200 Subject: [PATCH 138/151] [BEAM-9723] Add DLP integration transforms (#11566) [BEAM-9723] Add Google Cloud DLP integration transforms. * DLPDeidentifyText * DLPReidentifyText * DLPInspectText are now available. --- CHANGES.md | 4 + sdks/java/extensions/ml/build.gradle | 15 +- .../sdk/extensions/ml/BatchRequestForDLP.java | 113 +++++++ .../sdk/extensions/ml/DLPDeidentifyText.java | 282 +++++++++++++++++ .../sdk/extensions/ml/DLPInspectText.java | 240 +++++++++++++++ .../sdk/extensions/ml/DLPReidentifyText.java | 286 ++++++++++++++++++ .../sdk/extensions/ml/MapStringToDlpRow.java | 59 ++++ .../extensions/ml/BatchRequestForDlpTest.java | 63 ++++ .../extensions/ml/DLPDeidentifyTextTest.java | 101 +++++++ .../sdk/extensions/ml/DLPInspectTextTest.java | 101 +++++++ .../extensions/ml/DLPReidentifyTextTest.java | 101 +++++++ .../extensions/ml/DLPTextOperationsIT.java | 159 ++++++++++ .../extensions/ml/MapStringToDlpRowTest.java | 69 +++++ 13 files changed, 1591 insertions(+), 2 deletions(-) create mode 100644 sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java create mode 100644 sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java create mode 100644 sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java create mode 100644 sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java create mode 100644 sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRow.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyTextTest.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPInspectTextTest.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyTextTest.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.java create mode 100644 sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java diff --git a/CHANGES.md b/CHANGES.md index 0168d04216c6..d00b9c7fa1e9 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -93,6 +93,10 @@ * `--direct_num_workers=0` is supported for FnApi runner. It will set the number of threads/subprocesses to number of cores of the machine executing the pipeline ([BEAM-9443](https://issues.apache.org/jira/browse/BEAM-9443)). * Python SDK now has experimental support for SqlTransform ([BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603)). * Add OnWindowExpiration method to Stateful DoFn ([BEAM-1589](https://issues.apache.org/jira/browse/BEAM-1589)). +* Added PTransforms for Google Cloud DLP (Data Loss Prevention) services integration ([BEAM-9723](https://issues.apache.org/jira/browse/BEAM-9723)): + * Inspection of data, + * Deidentification of data, + * Reidentification of data. * Add a more complete I/O support matrix in the documentation site ([BEAM-9916](https://issues.apache.org/jira/browse/BEAM-9916)). * Upgrade Sphinx to 3.0.3 for building PyDoc. diff --git a/sdks/java/extensions/ml/build.gradle b/sdks/java/extensions/ml/build.gradle index 1f61c6a6da4a..c5e1ed3905fc 100644 --- a/sdks/java/extensions/ml/build.gradle +++ b/sdks/java/extensions/ml/build.gradle @@ -1,3 +1,5 @@ +import groovy.json.JsonOutput + /* * * * Licensed to the Apache Software Foundation (ASF) under one @@ -19,7 +21,7 @@ */ plugins { id 'org.apache.beam.module' } -applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.extensions.protobuf') +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.extensions.ml') description = 'Apache Beam :: SDKs :: Java :: Extensions :: ML' @@ -27,16 +29,25 @@ dependencies { compile project(path: ":sdks:java:core", configuration: "shadow") compile project(":sdks:java:expansion-service") compile 'com.google.cloud:google-cloud-video-intelligence:1.2.0' + compile 'com.google.cloud:google-cloud-dlp:1.1.4' compile 'com.google.cloud:google-cloud-language:1.99.4' + provided library.java.junit + testCompile project(path: ':sdks:java:core', configuration: 'shadowTest') testCompile project(path: ':sdks:java:core', configuration: 'shadowTest') testCompile library.java.mockito_core testCompile 'com.google.cloud:google-cloud-video-intelligence:1.2.0' + testCompile 'com.google.cloud:google-cloud-dlp:1.1.4' + testCompile project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntime") testCompile 'com.google.cloud:google-cloud-language:1.99.4' - testCompile library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly project(":runners:google-cloud-dataflow-java") } project.test { + def gcpProject = project.findProperty("gcpProject") ?: 'apache-beam-testing' include "**/**IT.class" + def pipelineOptions = [ + "--project=${gcpProject}" + ] + systemProperty "beamTestPipelineOptions", JsonOutput.toJson(pipelineOptions) } diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java new file mode 100644 index 000000000000..aabac4e0fcda --- /dev/null +++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.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.beam.sdk.extensions.ml; + +import com.google.privacy.dlp.v2.Table; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Batches input rows to reduce number of requests sent to Cloud DLP service. */ +@Experimental +class BatchRequestForDLP extends DoFn, KV>> { + public static final Logger LOG = LoggerFactory.getLogger(BatchRequestForDLP.class); + + private final Counter numberOfRowsBagged = + Metrics.counter(BatchRequestForDLP.class, "numberOfRowsBagged"); + + private final Integer batchSizeBytes; + + @StateId("elementsBag") + private final StateSpec>> elementsBag = StateSpecs.bag(); + + @TimerId("eventTimer") + private final TimerSpec eventTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + /** + * Constructs the batching DoFn. + * + * @param batchSize Desired batch size in bytes. + */ + public BatchRequestForDLP(Integer batchSize) { + this.batchSizeBytes = batchSize; + } + + @ProcessElement + public void process( + @Element KV element, + @StateId("elementsBag") BagState> elementsBag, + @TimerId("eventTimer") Timer eventTimer, + BoundedWindow w) { + elementsBag.add(element); + eventTimer.set(w.maxTimestamp()); + } + + /** + * Outputs the elements buffered in the elementsBag in batches of desired size. + * + * @param elementsBag element buffer. + * @param output Batched input elements. + */ + @OnTimer("eventTimer") + public void onTimer( + @StateId("elementsBag") BagState> elementsBag, + OutputReceiver>> output) { + if (elementsBag.read().iterator().hasNext()) { + String key = elementsBag.read().iterator().next().getKey(); + AtomicInteger bufferSize = new AtomicInteger(); + List rows = new ArrayList<>(); + elementsBag + .read() + .forEach( + element -> { + int elementSize = element.getValue().getSerializedSize(); + boolean clearBuffer = bufferSize.intValue() + elementSize > batchSizeBytes; + if (clearBuffer) { + LOG.debug( + "Clear buffer of {} bytes, Key {}", bufferSize.intValue(), element.getKey()); + numberOfRowsBagged.inc(rows.size()); + output.output(KV.of(element.getKey(), rows)); + rows.clear(); + bufferSize.set(0); + } + rows.add(element.getValue()); + bufferSize.getAndAdd(element.getValue().getSerializedSize()); + }); + if (!rows.isEmpty()) { + LOG.debug("Outputting remaining {} rows.", rows.size()); + numberOfRowsBagged.inc(rows.size()); + output.output(KV.of(key, rows)); + } + } + } +} diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java new file mode 100644 index 000000000000..0502950c70f6 --- /dev/null +++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java @@ -0,0 +1,282 @@ +/* + * 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.beam.sdk.extensions.ml; + +import com.google.auto.value.AutoValue; +import com.google.cloud.dlp.v2.DlpServiceClient; +import com.google.privacy.dlp.v2.ContentItem; +import com.google.privacy.dlp.v2.DeidentifyConfig; +import com.google.privacy.dlp.v2.DeidentifyContentRequest; +import com.google.privacy.dlp.v2.DeidentifyContentResponse; +import com.google.privacy.dlp.v2.FieldId; +import com.google.privacy.dlp.v2.InspectConfig; +import com.google.privacy.dlp.v2.ProjectName; +import com.google.privacy.dlp.v2.Table; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A {@link PTransform} connecting to Cloud DLP (https://cloud.google.com/dlp/docs/libraries) and + * deidentifying text according to provided settings. The transform supports both columnar delimited + * input data (eg. CSV) and unstructured input. + * + *

    If the headerColumns property is set and a sideinput with table headers is added to the + * PTransform, delimiter also should be set, else the results will be incorrect. If headerColumns is + * neither set nor passed as side input, input is assumed to be unstructured. + * + *

    Either deidentifyTemplateName (String) or deidentifyConfig {@link DeidentifyConfig} need to be + * set. inspectTemplateName and inspectConfig ({@link InspectConfig} are optional. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + * + *

    The transform consumes {@link KV} of {@link String}s (assumed to be filename as key and + * contents as value) and outputs {@link KV} of {@link String} (eg. filename) and {@link + * DeidentifyContentResponse}, which will contain {@link Table} of results for the user to consume. + */ +@Experimental +@AutoValue +public abstract class DLPDeidentifyText + extends PTransform< + PCollection>, PCollection>> { + + public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000; + + /** @return Template name for data inspection. */ + @Nullable + public abstract String getInspectTemplateName(); + + /** @return Template name for data deidentification. */ + @Nullable + public abstract String getDeidentifyTemplateName(); + + /** + * @return Configuration object for data inspection. If present, supersedes the template settings. + */ + @Nullable + public abstract InspectConfig getInspectConfig(); + + /** @return Configuration object for deidentification. If present, supersedes the template. */ + @Nullable + public abstract DeidentifyConfig getDeidentifyConfig(); + + /** @return List of column names if the input KV value is a delimited row. */ + @Nullable + public abstract PCollectionView> getHeaderColumns(); + + /** @return Delimiter to be used when splitting values from input strings into columns. */ + @Nullable + public abstract String getColumnDelimiter(); + + /** @return Size of input elements batch to be sent to Cloud DLP service in one request. */ + public abstract Integer getBatchSizeBytes(); + + /** @return ID of Google Cloud project to be used when deidentifying data. */ + public abstract String getProjectId(); + + @AutoValue.Builder + public abstract static class Builder { + /** @param inspectTemplateName Template name for data inspection. */ + public abstract Builder setInspectTemplateName(String inspectTemplateName); + + /** @param headerColumns List of column names if the input KV value is a delimited row. */ + public abstract Builder setHeaderColumns(PCollectionView> headerColumns); + + /** + * @param delimiter Delimiter to be used when splitting values from input strings into columns. + */ + public abstract Builder setColumnDelimiter(String delimiter); + + /** + * @param batchSize Size of input elements batch to be sent to Cloud DLP service in one request. + */ + public abstract Builder setBatchSizeBytes(Integer batchSize); + + /** @param projectId ID of Google Cloud project to be used when deidentifying data. */ + public abstract Builder setProjectId(String projectId); + + /** @param deidentifyTemplateName Template name for data deidentification. */ + public abstract Builder setDeidentifyTemplateName(String deidentifyTemplateName); + + /** + * @param inspectConfig Configuration object for data inspection. If present, supersedes the + * template settings. + */ + public abstract Builder setInspectConfig(InspectConfig inspectConfig); + + /** + * @param deidentifyConfig Configuration object for data deidentification. If present, + * supersedes the template settings. + */ + public abstract Builder setDeidentifyConfig(DeidentifyConfig deidentifyConfig); + + abstract DLPDeidentifyText autoBuild(); + + public DLPDeidentifyText build() { + DLPDeidentifyText dlpDeidentifyText = autoBuild(); + if (dlpDeidentifyText.getDeidentifyConfig() == null + && dlpDeidentifyText.getDeidentifyTemplateName() == null) { + throw new IllegalArgumentException( + "Either deidentifyConfig or deidentifyTemplateName need to be set!"); + } + if (dlpDeidentifyText.getBatchSizeBytes() > DLP_PAYLOAD_LIMIT_BYTES) { + throw new IllegalArgumentException( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLP_PAYLOAD_LIMIT_BYTES)); + } + if (dlpDeidentifyText.getColumnDelimiter() == null + && dlpDeidentifyText.getHeaderColumns() != null) { + throw new IllegalArgumentException( + "Column delimiter should be set if headers are present."); + } + if (dlpDeidentifyText.getHeaderColumns() == null + && dlpDeidentifyText.getColumnDelimiter() != null) { + throw new IllegalArgumentException( + "Column headers should be supplied when delimiter is present."); + } + return dlpDeidentifyText; + } + } + + public static DLPDeidentifyText.Builder newBuilder() { + return new AutoValue_DLPDeidentifyText.Builder(); + } + + /** + * The transform converts the contents of input PCollection into {@link Table.Row}s and then calls + * Cloud DLP service to perform the deidentification according to provided settings. + * + * @param input input PCollection + * @return PCollection after transformations + */ + @Override + public PCollection> expand( + PCollection> input) { + return input + .apply(ParDo.of(new MapStringToDlpRow(getColumnDelimiter()))) + .apply("Batch Contents", ParDo.of(new BatchRequestForDLP(getBatchSizeBytes()))) + .apply( + "DLPDeidentify", + ParDo.of( + new DeidentifyText( + getProjectId(), + getInspectTemplateName(), + getDeidentifyTemplateName(), + getInspectConfig(), + getDeidentifyConfig(), + getHeaderColumns()))); + } + + /** DoFn performing calls to Cloud DLP service on GCP. */ + static class DeidentifyText + extends DoFn>, KV> { + private final String projectId; + private final String inspectTemplateName; + private final String deidentifyTemplateName; + private final InspectConfig inspectConfig; + private final DeidentifyConfig deidentifyConfig; + private final PCollectionView> headerColumns; + private transient DeidentifyContentRequest.Builder requestBuilder; + private transient DlpServiceClient dlpServiceClient; + + @Setup + public void setup() throws IOException { + requestBuilder = + DeidentifyContentRequest.newBuilder().setParent(ProjectName.of(projectId).toString()); + if (inspectTemplateName != null) { + requestBuilder.setInspectTemplateName(inspectTemplateName); + } + if (inspectConfig != null) { + requestBuilder.setInspectConfig(inspectConfig); + } + if (deidentifyConfig != null) { + requestBuilder.setDeidentifyConfig(deidentifyConfig); + } + if (deidentifyTemplateName != null) { + requestBuilder.setDeidentifyTemplateName(deidentifyTemplateName); + } + dlpServiceClient = DlpServiceClient.create(); + } + + @Teardown + public void teardown() { + dlpServiceClient.close(); + } + + /** + * @param projectId ID of GCP project that should be used for deidentification. + * @param inspectTemplateName Template name for inspection. Optional. + * @param deidentifyTemplateName Template name for deidentification. Either this or + * deidentifyConfig is required. + * @param inspectConfig Configuration object for inspection. Optional. + * @param deidentifyConfig Deidentification config containing data transformations. Either this + * or deidentifyTemplateName is required. + * @param headerColumns Header row of the table if applicable. + */ + public DeidentifyText( + String projectId, + String inspectTemplateName, + String deidentifyTemplateName, + InspectConfig inspectConfig, + DeidentifyConfig deidentifyConfig, + PCollectionView> headerColumns) { + this.projectId = projectId; + this.inspectTemplateName = inspectTemplateName; + this.deidentifyTemplateName = deidentifyTemplateName; + this.inspectConfig = inspectConfig; + this.deidentifyConfig = deidentifyConfig; + this.headerColumns = headerColumns; + } + + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + String fileName = c.element().getKey(); + List dlpTableHeaders; + if (headerColumns != null) { + dlpTableHeaders = + c.sideInput(headerColumns).stream() + .map(header -> FieldId.newBuilder().setName(header).build()) + .collect(Collectors.toList()); + } else { + // handle unstructured input + dlpTableHeaders = new ArrayList<>(); + dlpTableHeaders.add(FieldId.newBuilder().setName("value").build()); + } + Table table = + Table.newBuilder() + .addAllHeaders(dlpTableHeaders) + .addAllRows(c.element().getValue()) + .build(); + ContentItem contentItem = ContentItem.newBuilder().setTable(table).build(); + this.requestBuilder.setItem(contentItem); + DeidentifyContentResponse response = + dlpServiceClient.deidentifyContent(this.requestBuilder.build()); + c.output(KV.of(fileName, response)); + } + } +} diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java new file mode 100644 index 000000000000..ff58674fbfcc --- /dev/null +++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java @@ -0,0 +1,240 @@ +/* + * 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.beam.sdk.extensions.ml; + +import com.google.auto.value.AutoValue; +import com.google.cloud.dlp.v2.DlpServiceClient; +import com.google.privacy.dlp.v2.ContentItem; +import com.google.privacy.dlp.v2.FieldId; +import com.google.privacy.dlp.v2.InspectConfig; +import com.google.privacy.dlp.v2.InspectContentRequest; +import com.google.privacy.dlp.v2.InspectContentResponse; +import com.google.privacy.dlp.v2.ProjectName; +import com.google.privacy.dlp.v2.Table; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A {@link PTransform} connecting to Cloud DLP (https://cloud.google.com/dlp/docs/libraries) and + * inspecting text for identifying data according to provided settings. The transform supports both + * delimited columnar input data (eg. CSV) and unstructured input. + * + *

    If the headerColumns property is set and a sideinput with table headers is added to the + * PTransform, delimiter also should be set, else the results will be incorrect. If headerColumns is + * neither set nor passed as sideinput, input is assumed to be unstructured. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + * + *

    The transform consumes {@link KV} of {@link String}s (assumed to be filename as key and + * contents as value) and outputs {@link KV} of {@link String} (eg. filename) and {@link + * InspectContentResponse}, which will contain a list of {@link + * com.google.privacy.dlp.v2.InspectResult} for the user to consume. + * + *

    Either inspectTemplateName (String) or inspectConfig {@link InspectConfig} need to be set. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + */ +@Experimental +@AutoValue +public abstract class DLPInspectText + extends PTransform< + PCollection>, PCollection>> { + + public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000; + + /** @return Template name for data inspection. */ + @Nullable + public abstract String getInspectTemplateName(); + + /** + * @return Configuration object for data inspection. If present, supersedes the template settings. + */ + @Nullable + public abstract InspectConfig getInspectConfig(); + + /** @return Size of input elements batch to be sent to Cloud DLP service in one request. */ + public abstract Integer getBatchSizeBytes(); + + /** @return ID of Google Cloud project to be used when deidentifying data. */ + public abstract String getProjectId(); + + /** @return Delimiter to be used when splitting values from input strings into columns. */ + @Nullable + public abstract String getColumnDelimiter(); + + /** @return List of column names if the input KV value is a delimited row. */ + @Nullable + public abstract PCollectionView> getHeaderColumns(); + + @AutoValue.Builder + public abstract static class Builder { + /** @param inspectTemplateName Template name for data inspection. */ + public abstract Builder setInspectTemplateName(String inspectTemplateName); + + /** + * @param inspectConfig Configuration object for data inspection. If present, supersedes the + * template settings. + */ + public abstract Builder setInspectConfig(InspectConfig inspectConfig); + + /** + * @param batchSize Size of input elements batch to be sent to Cloud DLP service in one request. + */ + public abstract Builder setBatchSizeBytes(Integer batchSize); + + /** @param projectId ID of Google Cloud project to be used when deidentifying data. */ + public abstract Builder setProjectId(String projectId); + + /** + * @param delimiter Delimiter to be used when splitting values from input strings into columns. + */ + public abstract Builder setColumnDelimiter(String delimiter); + + /** @param headerColumns List of column names if the input KV value is a delimited row. */ + public abstract Builder setHeaderColumns(PCollectionView> headerColumns); + + abstract DLPInspectText autoBuild(); + + public DLPInspectText build() { + DLPInspectText inspectText = autoBuild(); + if (inspectText.getInspectTemplateName() == null && inspectText.getInspectConfig() == null) { + throw new IllegalArgumentException( + "Either inspectTemplateName or inspectConfig must be supplied!"); + } + if (inspectText.getBatchSizeBytes() > DLP_PAYLOAD_LIMIT_BYTES) { + throw new IllegalArgumentException( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLP_PAYLOAD_LIMIT_BYTES)); + } + if (inspectText.getColumnDelimiter() == null && inspectText.getHeaderColumns() != null) { + throw new IllegalArgumentException( + "Column delimiter should be set if headers are present."); + } + if (inspectText.getHeaderColumns() == null && inspectText.getColumnDelimiter() != null) { + throw new IllegalArgumentException( + "Column headers should be supplied when delimiter is present."); + } + return inspectText; + } + } + + public static Builder newBuilder() { + return new AutoValue_DLPInspectText.Builder(); + } + + /** + * The transform converts the contents of input PCollection into {@link Table.Row}s and then calls + * Cloud DLP service to perform the data inspection according to provided settings. + * + * @param input input PCollection + * @return PCollection after transformations + */ + @Override + public PCollection> expand( + PCollection> input) { + return input + .apply(ParDo.of(new MapStringToDlpRow(getColumnDelimiter()))) + .apply("Batch Contents", ParDo.of(new BatchRequestForDLP(getBatchSizeBytes()))) + .apply( + "DLPInspect", + ParDo.of( + new InspectData( + getProjectId(), + getInspectTemplateName(), + getInspectConfig(), + getHeaderColumns()))); + } + + /** Performs calls to Cloud DLP service on GCP to inspect input data. */ + static class InspectData + extends DoFn>, KV> { + private final String projectId; + private final String inspectTemplateName; + private final InspectConfig inspectConfig; + private final PCollectionView> headerColumns; + private transient DlpServiceClient dlpServiceClient; + private transient InspectContentRequest.Builder requestBuilder; + + /** + * @param projectId ID of GCP project that should be used for data inspection. + * @param inspectTemplateName Template name for inspection. + * @param inspectConfig Configuration object for inspection. + * @param headerColumns Header row of the table if applicable. + */ + public InspectData( + String projectId, + String inspectTemplateName, + InspectConfig inspectConfig, + PCollectionView> headerColumns) { + this.projectId = projectId; + this.inspectTemplateName = inspectTemplateName; + this.inspectConfig = inspectConfig; + this.headerColumns = headerColumns; + } + + @Setup + public void setup() throws IOException { + this.requestBuilder = + InspectContentRequest.newBuilder().setParent(ProjectName.of(this.projectId).toString()); + if (inspectTemplateName != null) { + requestBuilder.setInspectTemplateName(this.inspectTemplateName); + } + if (inspectConfig != null) { + requestBuilder.setInspectConfig(inspectConfig); + } + dlpServiceClient = DlpServiceClient.create(); + } + + @Teardown + public void teardown() { + dlpServiceClient.close(); + } + + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + List tableHeaders; + if (headerColumns != null) { + tableHeaders = + c.sideInput(headerColumns).stream() + .map(header -> FieldId.newBuilder().setName(header).build()) + .collect(Collectors.toList()); + } else { + tableHeaders = new ArrayList<>(); + tableHeaders.add(FieldId.newBuilder().setName("value").build()); + } + Table table = + Table.newBuilder().addAllHeaders(tableHeaders).addAllRows(c.element().getValue()).build(); + ContentItem contentItem = ContentItem.newBuilder().setTable(table).build(); + this.requestBuilder.setItem(contentItem); + InspectContentResponse response = + dlpServiceClient.inspectContent(this.requestBuilder.build()); + c.output(KV.of(c.element().getKey(), response)); + } + } +} diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java new file mode 100644 index 000000000000..5dc20cbc3697 --- /dev/null +++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java @@ -0,0 +1,286 @@ +/* + * 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.beam.sdk.extensions.ml; + +import com.google.auto.value.AutoValue; +import com.google.cloud.dlp.v2.DlpServiceClient; +import com.google.privacy.dlp.v2.ContentItem; +import com.google.privacy.dlp.v2.DeidentifyConfig; +import com.google.privacy.dlp.v2.FieldId; +import com.google.privacy.dlp.v2.InspectConfig; +import com.google.privacy.dlp.v2.ProjectName; +import com.google.privacy.dlp.v2.ReidentifyContentRequest; +import com.google.privacy.dlp.v2.ReidentifyContentResponse; +import com.google.privacy.dlp.v2.Table; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A {@link PTransform} connecting to Cloud DLP (https://cloud.google.com/dlp/docs/libraries) and + * inspecting text for identifying data according to provided settings. + * + *

    The transform supports both delimited columnar input data and unstructured input. + * + *

    If the headerColumns property is set and a sideinput with headers is added to the PTransform, + * delimiter also should be set, else the results will be incorrect. If headerColumns is neither set + * nor passed as sideinput, input is assumed to be unstructured. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + * + *

    The transform consumes {@link KV} of {@link String}s (assumed to be filename as key and + * contents as value) and outputs {@link KV} of {@link String} (eg. filename) and {@link + * ReidentifyContentResponse}, which will contain {@link Table} of results for the user to consume. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + * + *

    Either reidentifyTemplateName {@link String} or reidentifyConfig {@link DeidentifyConfig} need + * to be set. inspectConfig {@link InspectConfig} and inspectTemplateName {@link String} are + * optional. + * + *

    Batch size defines how big are batches sent to DLP at once in bytes. + */ +@Experimental +@AutoValue +public abstract class DLPReidentifyText + extends PTransform< + PCollection>, PCollection>> { + + public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000; + + /** @return Template name for data inspection. */ + @Nullable + public abstract String getInspectTemplateName(); + + /** @return Template name for data reidentification. */ + @Nullable + public abstract String getReidentifyTemplateName(); + + /** + * @return Configuration object for data inspection. If present, supersedes the template settings. + */ + @Nullable + public abstract InspectConfig getInspectConfig(); + + /** @return Configuration object for reidentification. If present, supersedes the template. */ + @Nullable + public abstract DeidentifyConfig getReidentifyConfig(); + + /** @return Delimiter to be used when splitting values from input strings into columns. */ + @Nullable + public abstract String getColumnDelimiter(); + + /** @return List of column names if the input KV value is a delimited row. */ + @Nullable + public abstract PCollectionView> getHeaderColumns(); + + /** @return Size of input elements batch to be sent to Cloud DLP service in one request. */ + public abstract Integer getBatchSizeBytes(); + + /** @return ID of Google Cloud project to be used when deidentifying data. */ + public abstract String getProjectId(); + + @AutoValue.Builder + public abstract static class Builder { + /** @param inspectTemplateName Template name for data inspection. */ + public abstract Builder setInspectTemplateName(String inspectTemplateName); + + /** + * @param inspectConfig Configuration object for data inspection. If present, supersedes the + * template settings. + */ + public abstract Builder setInspectConfig(InspectConfig inspectConfig); + + /** + * @param reidentifyConfig Configuration object for data deidentification. If present, + * supersedes the template settings. + */ + public abstract Builder setReidentifyConfig(DeidentifyConfig reidentifyConfig); + + /** @param reidentifyTemplateName Template name for data deidentification. */ + public abstract Builder setReidentifyTemplateName(String reidentifyTemplateName); + + /** + * @param batchSize Size of input elements batch to be sent to Cloud DLP service in one request. + */ + public abstract Builder setBatchSizeBytes(Integer batchSize); + /** @param headerColumns List of column names if the input KV value is a delimited row. */ + public abstract Builder setHeaderColumns(PCollectionView> headerColumns); + + /** + * @param delimiter Delimiter to be used when splitting values from input strings into columns. + */ + public abstract Builder setColumnDelimiter(String delimiter); + + /** @param projectId ID of Google Cloud project to be used when deidentifying data. */ + public abstract Builder setProjectId(String projectId); + + abstract DLPReidentifyText autoBuild(); + + public DLPReidentifyText build() { + DLPReidentifyText dlpReidentifyText = autoBuild(); + if (dlpReidentifyText.getReidentifyConfig() == null + && dlpReidentifyText.getReidentifyTemplateName() == null) { + throw new IllegalArgumentException( + "Either reidentifyConfig or reidentifyTemplateName need to be set!"); + } + if (dlpReidentifyText.getBatchSizeBytes() > DLP_PAYLOAD_LIMIT_BYTES) { + throw new IllegalArgumentException( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLP_PAYLOAD_LIMIT_BYTES)); + } + if (dlpReidentifyText.getColumnDelimiter() == null + && dlpReidentifyText.getHeaderColumns() != null) { + throw new IllegalArgumentException( + "Column delimiter should be set if headers are present."); + } + if (dlpReidentifyText.getHeaderColumns() == null + && dlpReidentifyText.getColumnDelimiter() != null) { + throw new IllegalArgumentException( + "Column headers should be supplied when delimiter is present."); + } + return dlpReidentifyText; + } + } + + public static DLPReidentifyText.Builder newBuilder() { + return new AutoValue_DLPReidentifyText.Builder(); + } + + /** + * The transform converts the contents of input PCollection into {@link Table.Row}s and then calls + * Cloud DLP service to perform the reidentification according to provided settings. + * + * @param input input PCollection + * @return PCollection after transformations + */ + @Override + public PCollection> expand( + PCollection> input) { + return input + .apply(ParDo.of(new MapStringToDlpRow(getColumnDelimiter()))) + .apply("Batch Contents", ParDo.of(new BatchRequestForDLP(getBatchSizeBytes()))) + .apply( + "DLPReidentify", + ParDo.of( + new ReidentifyText( + getProjectId(), + getInspectTemplateName(), + getReidentifyTemplateName(), + getInspectConfig(), + getReidentifyConfig(), + getHeaderColumns()))); + } + + /** Performs the calls to Cloud DLP service on GCP. */ + static class ReidentifyText + extends DoFn>, KV> { + private final String projectId; + private final String inspectTemplateName; + private final String reidentifyTemplateName; + private final InspectConfig inspectConfig; + private final DeidentifyConfig reidentifyConfig; + private final PCollectionView> headerColumns; + private transient ReidentifyContentRequest.Builder requestBuilder; + private transient DlpServiceClient dlpServiceClient; + + @Setup + public void setup() throws IOException { + requestBuilder = + ReidentifyContentRequest.newBuilder().setParent(ProjectName.of(projectId).toString()); + if (inspectTemplateName != null) { + requestBuilder.setInspectTemplateName(inspectTemplateName); + } + if (inspectConfig != null) { + requestBuilder.setInspectConfig(inspectConfig); + } + if (reidentifyConfig != null) { + requestBuilder.setReidentifyConfig(reidentifyConfig); + } + if (reidentifyTemplateName != null) { + requestBuilder.setReidentifyTemplateName(reidentifyTemplateName); + } + dlpServiceClient = DlpServiceClient.create(); + } + + @Teardown + public void teardown() { + dlpServiceClient.close(); + } + + /** + * @param projectId ID of GCP project that should be used for deidentification. + * @param inspectTemplateName Template name for inspection. Optional. + * @param reidentifyTemplateName Template name for reidentification. Either this or + * reidentifyConfig is required. + * @param inspectConfig Configuration object for inspection. Optional. + * @param reidentifyConfig Reidentification config containing data transformations. Either this + * or reidentifyTemplateName is required. + * @param headerColumns Header row of the table if applicable. + */ + public ReidentifyText( + String projectId, + String inspectTemplateName, + String reidentifyTemplateName, + InspectConfig inspectConfig, + DeidentifyConfig reidentifyConfig, + PCollectionView> headerColumns) { + this.projectId = projectId; + this.inspectTemplateName = inspectTemplateName; + this.reidentifyTemplateName = reidentifyTemplateName; + this.inspectConfig = inspectConfig; + this.reidentifyConfig = reidentifyConfig; + this.headerColumns = headerColumns; + } + + @ProcessElement + public void processElement(ProcessContext context) throws IOException { + List tableHeaders; + if (headerColumns != null) { + tableHeaders = + context.sideInput(headerColumns).stream() + .map(header -> FieldId.newBuilder().setName(header).build()) + .collect(Collectors.toList()); + } else { + // handle unstructured input. + tableHeaders = new ArrayList<>(); + tableHeaders.add(FieldId.newBuilder().setName("value").build()); + } + Table table = + Table.newBuilder() + .addAllHeaders(tableHeaders) + .addAllRows(context.element().getValue()) + .build(); + ContentItem contentItem = ContentItem.newBuilder().setTable(table).build(); + this.requestBuilder.setItem(contentItem); + ReidentifyContentResponse response = + dlpServiceClient.reidentifyContent(requestBuilder.build()); + context.output(KV.of(context.element().getKey(), response)); + } + } +} diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRow.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRow.java new file mode 100644 index 000000000000..944656c59fb1 --- /dev/null +++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRow.java @@ -0,0 +1,59 @@ +/* + * 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.beam.sdk.extensions.ml; + +import com.google.privacy.dlp.v2.Table; +import com.google.privacy.dlp.v2.Value; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; + +/** + * Maps {@link KV}s of {@link String}s into KV<{@link String}, {@link Table.Row}> for further + * processing in the DLP transforms. + * + *

    If a delimiter of values isn't provided, input is assumed to be unstructured and the input KV + * value is saved in a single column of output {@link Table.Row}. + */ +class MapStringToDlpRow extends DoFn, KV> { + private final String delimiter; + + /** + * @param delimiter Delimiter of values in the delimited value row that may be in the value of + * input KV. + */ + public MapStringToDlpRow(String delimiter) { + this.delimiter = delimiter; + } + + @ProcessElement + public void processElement(ProcessContext context) { + Table.Row.Builder rowBuilder = Table.Row.newBuilder(); + String line = Objects.requireNonNull(context.element().getValue()); + if (delimiter != null) { + List values = Arrays.asList(line.split(delimiter)); + values.forEach( + value -> rowBuilder.addValues(Value.newBuilder().setStringValue(value).build())); + } else { + rowBuilder.addValues(Value.newBuilder().setStringValue(line).build()); + } + context.output(KV.of(context.element().getKey(), rowBuilder.build())); + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java new file mode 100644 index 000000000000..3e9a74178929 --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java @@ -0,0 +1,63 @@ +/* + * 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.beam.sdk.extensions.ml; + +import static org.junit.Assert.assertEquals; + +import com.google.privacy.dlp.v2.Table; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BatchRequestForDlpTest { + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void batchesRequests() { + PCollection>> batchedRows = + testPipeline + .apply(Create.of(KV.of("key", "value1"), KV.of("key", "value2"))) + .apply(ParDo.of(new MapStringToDlpRow(null))) + .apply(ParDo.of(new BatchRequestForDLP(524000))); + PAssert.that(batchedRows).satisfies(new VerifyPCollectionSize()); + testPipeline.run().waitUntilFinish(); + } + + private static class VerifyPCollectionSize + implements SerializableFunction>>, Void> { + @Override + public Void apply(Iterable>> input) { + List>> itemList = new ArrayList<>(); + input.forEach(itemList::add); + assertEquals(1, itemList.size()); + return null; + } + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyTextTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyTextTest.java new file mode 100644 index 000000000000..c2e5741d81db --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyTextTest.java @@ -0,0 +1,101 @@ +/* + * 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.beam.sdk.extensions.ml; + +import static org.junit.Assert.assertThrows; + +import java.util.List; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PCollectionView; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DLPDeidentifyTextTest { + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + private static final Integer BATCH_SIZE_SMALL = 200; + private static final String DELIMITER = ";"; + private static final String TEMPLATE_NAME = "test_template"; + private static final String PROJECT_ID = "test_id"; + + @Test + public void throwsExceptionWhenDeidentifyConfigAndTemplatesAreEmpty() { + assertThrows( + "Either deidentifyConfig or deidentifyTemplateName need to be set!", + IllegalArgumentException.class, + () -> + DLPDeidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsNullAndHeadersAreSet() { + PCollectionView> header = + testPipeline.apply(Create.of("header")).apply(View.asList()); + assertThrows( + "Column delimiter should be set if headers are present.", + IllegalArgumentException.class, + () -> + DLPDeidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setDeidentifyTemplateName(TEMPLATE_NAME) + .setHeaderColumns(header) + .build()); + testPipeline.run().waitUntilFinish(); + } + + @Test + public void throwsExceptionWhenBatchSizeIsTooLarge() { + assertThrows( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLPDeidentifyText.DLP_PAYLOAD_LIMIT_BYTES), + IllegalArgumentException.class, + () -> + DLPDeidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(Integer.MAX_VALUE) + .setDeidentifyTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsSetAndHeadersAreNot() { + assertThrows( + "Column headers should be supplied when delimiter is present.", + IllegalArgumentException.class, + () -> + DLPDeidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setDeidentifyTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPInspectTextTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPInspectTextTest.java new file mode 100644 index 000000000000..46c8fa3a3096 --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPInspectTextTest.java @@ -0,0 +1,101 @@ +/* + * 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.beam.sdk.extensions.ml; + +import static org.junit.Assert.assertThrows; + +import java.util.List; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PCollectionView; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DLPInspectTextTest { + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + private static final Integer BATCH_SIZE_SMALL = 200; + private static final String DELIMITER = ";"; + private static final String TEMPLATE_NAME = "test_template"; + private static final String PROJECT_ID = "test_id"; + + @Test + public void throwsExceptionWhenDeidentifyConfigAndTemplatesAreEmpty() { + assertThrows( + "Either inspectTemplateName or inspectConfig must be supplied!", + IllegalArgumentException.class, + () -> + DLPInspectText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsNullAndHeadersAreSet() { + PCollectionView> header = + testPipeline.apply(Create.of("header")).apply(View.asList()); + assertThrows( + "Column delimiter should be set if headers are present.", + IllegalArgumentException.class, + () -> + DLPInspectText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setInspectTemplateName(TEMPLATE_NAME) + .setHeaderColumns(header) + .build()); + testPipeline.run().waitUntilFinish(); + } + + @Test + public void throwsExceptionWhenBatchSizeIsTooLarge() { + assertThrows( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLPInspectText.DLP_PAYLOAD_LIMIT_BYTES), + IllegalArgumentException.class, + () -> + DLPInspectText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(Integer.MAX_VALUE) + .setInspectTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsSetAndHeadersAreNot() { + assertThrows( + "Column headers should be supplied when delimiter is present.", + IllegalArgumentException.class, + () -> + DLPInspectText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setInspectTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyTextTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyTextTest.java new file mode 100644 index 000000000000..9fc0f0a9d7a7 --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyTextTest.java @@ -0,0 +1,101 @@ +/* + * 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.beam.sdk.extensions.ml; + +import static org.junit.Assert.assertThrows; + +import java.util.List; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PCollectionView; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DLPReidentifyTextTest { + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + private static final Integer BATCH_SIZE_SMALL = 200; + private static final String DELIMITER = ";"; + private static final String TEMPLATE_NAME = "test_template"; + private static final String PROJECT_ID = "test_id"; + + @Test + public void throwsExceptionWhenDeidentifyConfigAndTemplatesAreEmpty() { + assertThrows( + "Either reidentifyConfig or reidentifyTemplateName need to be set!", + IllegalArgumentException.class, + () -> + DLPReidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsNullAndHeadersAreSet() { + PCollectionView> header = + testPipeline.apply(Create.of("header")).apply(View.asList()); + assertThrows( + "Column delimiter should be set if headers are present.", + IllegalArgumentException.class, + () -> + DLPReidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setReidentifyTemplateName(TEMPLATE_NAME) + .setHeaderColumns(header) + .build()); + testPipeline.run().waitUntilFinish(); + } + + @Test + public void throwsExceptionWhenBatchSizeIsTooLarge() { + assertThrows( + String.format( + "Batch size is too large! It should be smaller or equal than %d.", + DLPDeidentifyText.DLP_PAYLOAD_LIMIT_BYTES), + IllegalArgumentException.class, + () -> + DLPReidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(Integer.MAX_VALUE) + .setReidentifyTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } + + @Test + public void throwsExceptionWhenDelimiterIsSetAndHeadersAreNot() { + assertThrows( + "Column headers should be supplied when delimiter is present.", + IllegalArgumentException.class, + () -> + DLPReidentifyText.newBuilder() + .setProjectId(PROJECT_ID) + .setBatchSizeBytes(BATCH_SIZE_SMALL) + .setReidentifyTemplateName(TEMPLATE_NAME) + .setColumnDelimiter(DELIMITER) + .build()); + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.java new file mode 100644 index 000000000000..0ebbfb7b0008 --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.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.beam.sdk.extensions.ml; + +import static org.junit.Assert.assertTrue; + +import com.google.privacy.dlp.v2.CharacterMaskConfig; +import com.google.privacy.dlp.v2.DeidentifyConfig; +import com.google.privacy.dlp.v2.DeidentifyContentResponse; +import com.google.privacy.dlp.v2.FieldId; +import com.google.privacy.dlp.v2.Finding; +import com.google.privacy.dlp.v2.InfoType; +import com.google.privacy.dlp.v2.InfoTypeTransformations; +import com.google.privacy.dlp.v2.InspectConfig; +import com.google.privacy.dlp.v2.InspectContentResponse; +import com.google.privacy.dlp.v2.Likelihood; +import com.google.privacy.dlp.v2.PrimitiveTransformation; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DLPTextOperationsIT { + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + private static final String IDENTIFYING_TEXT = "mary.sue@example.com"; + private static InfoType emailAddress = InfoType.newBuilder().setName("EMAIL_ADDRESS").build(); + private static final InspectConfig inspectConfig = + InspectConfig.newBuilder() + .addInfoTypes(emailAddress) + .setMinLikelihood(Likelihood.LIKELY) + .build(); + + @Test + public void inspectsText() { + String projectId = testPipeline.getOptions().as(GcpOptions.class).getProject(); + PCollection> inspectionResult = + testPipeline + .apply(Create.of(KV.of("", IDENTIFYING_TEXT))) + .apply( + DLPInspectText.newBuilder() + .setBatchSizeBytes(524000) + .setProjectId(projectId) + .setInspectConfig(inspectConfig) + .build()); + PAssert.that(inspectionResult).satisfies(new VerifyInspectionResult()); + testPipeline.run().waitUntilFinish(); + } + + @Test + public void deidentifiesText() { + String projectId = testPipeline.getOptions().as(GcpOptions.class).getProject(); + + PCollection> deidentificationResult = + testPipeline + .apply(Create.of(KV.of("", IDENTIFYING_TEXT))) + .apply( + DLPDeidentifyText.newBuilder() + .setBatchSizeBytes(524000) + .setProjectId(projectId) + .setDeidentifyConfig(getDeidentifyConfig()) + .build()); + PAssert.that(deidentificationResult) + .satisfies(new VerifyDeidentificationResult("####################")); + testPipeline.run().waitUntilFinish(); + } + + private DeidentifyConfig getDeidentifyConfig() { + CharacterMaskConfig characterMaskConfig = + CharacterMaskConfig.newBuilder().setMaskingCharacter("#").build(); + PrimitiveTransformation primitiveTransformation = + PrimitiveTransformation.newBuilder().setCharacterMaskConfig(characterMaskConfig).build(); + InfoTypeTransformations.InfoTypeTransformation infoTypeTransformation = + InfoTypeTransformations.InfoTypeTransformation.newBuilder() + .addInfoTypes(emailAddress) + .setPrimitiveTransformation(primitiveTransformation) + .build(); + return DeidentifyConfig.newBuilder() + .setInfoTypeTransformations( + InfoTypeTransformations.newBuilder().addTransformations(infoTypeTransformation).build()) + .build(); + } + + private static class VerifyInspectionResult + implements SerializableFunction>, Void> { + @Override + public Void apply(Iterable> input) { + List matches = new ArrayList<>(); + input.forEach( + item -> { + List resultList = item.getValue().getResult().getFindingsList(); + matches.add( + resultList.stream() + .anyMatch(finding -> finding.getInfoType().equals(emailAddress))); + }); + assertTrue(matches.contains(Boolean.TRUE)); + return null; + } + } + + private static class VerifyDeidentificationResult + implements SerializableFunction>, Void> { + private final String expectedValue; + + public VerifyDeidentificationResult(String expectedValue) { + this.expectedValue = expectedValue; + } + + @Override + public Void apply(Iterable> input) { + List matches = new ArrayList<>(); + input.forEach( + item -> { + item.getValue() + .getItem() + .getTable() + .getRowsList() + .forEach( + row -> + matches.add( + row.getValuesList().stream() + .anyMatch(value -> value.getStringValue().equals(expectedValue)))); + assertTrue( + item.getValue() + .getItem() + .getTable() + .getHeadersList() + .contains(FieldId.newBuilder().setName("value").build())); + }); + assertTrue(matches.contains(Boolean.TRUE)); + return null; + } + } +} diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java new file mode 100644 index 000000000000..577a5dc7ed34 --- /dev/null +++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java @@ -0,0 +1,69 @@ +/* + * 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.beam.sdk.extensions.ml; + +import com.google.privacy.dlp.v2.Table; +import com.google.privacy.dlp.v2.Value; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class MapStringToDlpRowTest { + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void mapsStringToRow() { + PCollection> rowCollection = + testPipeline + .apply(Create.of(KV.of("key", "value"))) + .apply(ParDo.of(new MapStringToDlpRow(null))); + PAssert.that(rowCollection) + .containsInAnyOrder( + KV.of( + "key", + Table.Row.newBuilder() + .addValues(Value.newBuilder().setStringValue("value").build()) + .build())); + testPipeline.run().waitUntilFinish(); + } + + @Test + public void mapsDelimitedStringToRow() { + PCollection> rowCollection = + testPipeline + .apply(Create.of(KV.of("key", "value,secondValue"))) + .apply(ParDo.of(new MapStringToDlpRow(","))); + PAssert.that(rowCollection) + .containsInAnyOrder( + KV.of( + "key", + Table.Row.newBuilder() + .addValues(Value.newBuilder().setStringValue("value").build()) + .addValues(Value.newBuilder().setStringValue("secondValue").build()) + .build())); + testPipeline.run().waitUntilFinish(); + } +} From 006b8ceafab48fbbbe84eeceb723b0fd8a4950ae Mon Sep 17 00:00:00 2001 From: Piotr Szuberski Date: Tue, 2 Jun 2020 14:47:20 +0200 Subject: [PATCH 139/151] [BEAM-10171] Add Kinesis, Spanner, Jdbc IOs to the portability roadmap --- website/www/site/content/en/roadmap/connectors-multi-sdk.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/www/site/content/en/roadmap/connectors-multi-sdk.md b/website/www/site/content/en/roadmap/connectors-multi-sdk.md index 55d11fd73d32..500c44c58545 100644 --- a/website/www/site/content/en/roadmap/connectors-multi-sdk.md +++ b/website/www/site/content/en/roadmap/connectors-multi-sdk.md @@ -76,8 +76,11 @@ Work related to making cross-language transforms available on Direct runner Ongoing and planned work related to making existing connectors/transforms available to other SDKs through the cross-language transforms framework. +* Java JdbcIO - In progress - [BEAM-10135](https://issues.apache.org/jira/browse/BEAM-10135), [BEAM-10136](https://issues.apache.org/jira/browse/BEAM-10136) * Java KafkaIO - completed - [BEAM-7029](https://issues.apache.org/jira/browse/BEAM-7029) +* Java KinesisIO - In progress - [BEAM-10137](https://issues.apache.org/jira/browse/BEAM-10137), [BEAM-10138](https://issues.apache.org/jira/browse/BEAM-10138) * Java PubSubIO - In progress - [BEAM-7738](https://issues.apache.org/jira/browse/BEAM-7738) +* Java SpannerIO - In progress - [BEAM-10139](https://issues.apache.org/jira/browse/BEAM-10139), [BEAM-10140](https://issues.apache.org/jira/browse/BEAM-10140) * Java SQL - completed - [BEAM-8603](https://issues.apache.org/jira/browse/BEAM-8603) ### Portable Beam schema From 0c451363db15126025d2b558213945c0bc0071b3 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Fri, 29 May 2020 16:43:22 +0200 Subject: [PATCH 140/151] [BEAM-8850,BEAM-8849] FlinkBatchTranslationContext respects input parameters during lookups. Add support for a multiple translation variants for a single urn. --- .../flink/FlinkBatchPipelineTranslator.java | 15 ++- .../flink/FlinkBatchTransformTranslators.java | 52 ++++++---- .../flink/FlinkBatchTranslationContext.java | 61 +++++++----- .../utils/CountingPipelineVisitor.java | 47 +++++++++ .../utils/LookupPipelineVisitor.java | 98 +++++++++++++++++++ 5 files changed, 221 insertions(+), 52 deletions(-) create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java index 772c9c9a8dce..7286713019ef 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java @@ -46,6 +46,7 @@ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions op @Override @SuppressWarnings("rawtypes, unchecked") public void translate(Pipeline pipeline) { + batchContext.init(pipeline); super.translate(pipeline); // terminate dangling DataSets @@ -63,7 +64,7 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.info("{} enterCompositeTransform- {}", genSpaces(this.depth), node.getFullName()); this.depth++; - BatchTransformTranslator translator = getTranslator(node); + BatchTransformTranslator translator = getTranslator(node, batchContext); if (translator != null) { applyBatchTransform(node.getTransform(), node, translator); @@ -88,7 +89,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); BatchTransformTranslator translator = - FlinkBatchTransformTranslators.getTranslator(transform); + FlinkBatchTransformTranslators.getTranslator(transform, batchContext); if (translator == null) { String transformUrn = PTransformTranslation.urnForTransform(transform); throw new UnsupportedOperationException( @@ -115,11 +116,17 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { /** A translator of a {@link PTransform}. */ public interface BatchTransformTranslator { + + default boolean canTranslate(TransformT transform, FlinkBatchTranslationContext context) { + return true; + } + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } /** Returns a translator for the given node, if it is possible, otherwise null. */ - private static BatchTransformTranslator getTranslator(TransformHierarchy.Node node) { + private static BatchTransformTranslator getTranslator( + TransformHierarchy.Node node, FlinkBatchTranslationContext context) { @Nullable PTransform transform = node.getTransform(); // Root of the graph is null @@ -127,6 +134,6 @@ private static BatchTransformTranslator getTranslator(TransformHierarchy.Node return null; } - return FlinkBatchTransformTranslators.getTranslator(transform); + return FlinkBatchTransformTranslators.getTranslator(transform, context); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 8e3b623850be..3f5104d1b7c3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -105,40 +105,50 @@ class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map + private static final Map> TRANSLATORS = new HashMap<>(); - static { - TRANSLATORS.put(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslatorBatch()); + private static void registerTranslator( + String urn, FlinkBatchPipelineTranslator.BatchTransformTranslator translator) { + if (!TRANSLATORS.containsKey(urn)) { + TRANSLATORS.put(urn, new ArrayList<>()); + } + TRANSLATORS.get(urn).add(translator); + } - TRANSLATORS.put( + static { + registerTranslator(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslatorBatch()); + registerTranslator( PTransformTranslation.CREATE_VIEW_TRANSFORM_URN, new CreatePCollectionViewTranslatorBatch()); - - TRANSLATORS.put( + registerTranslator( PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, new CombinePerKeyTranslatorBatch()); - TRANSLATORS.put( + registerTranslator( PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslatorBatch()); - TRANSLATORS.put(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch()); - - TRANSLATORS.put( + registerTranslator(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch()); + registerTranslator( PTransformTranslation.FLATTEN_TRANSFORM_URN, new FlattenPCollectionTranslatorBatch()); - - TRANSLATORS.put( + registerTranslator( PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, new WindowAssignTranslatorBatch()); - - TRANSLATORS.put(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch()); - - TRANSLATORS.put(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch()); + registerTranslator(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch()); + registerTranslator(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch()); } + @SuppressWarnings("unchecked") static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator( - PTransform transform) { - @Nullable String urn = PTransformTranslation.urnForTransformOrNull(transform); - return urn == null ? null : TRANSLATORS.get(urn); + PTransform transform, FlinkBatchTranslationContext context) { + @Nullable final String urn = PTransformTranslation.urnForTransformOrNull(transform); + if (urn != null && TRANSLATORS.containsKey(urn)) { + for (FlinkBatchPipelineTranslator.BatchTransformTranslator translator : + TRANSLATORS.get(urn)) { + if (translator.canTranslate(transform, context)) { + return translator; + } + } + } + return null; } - @SuppressWarnings("unchecked") private static String getCurrentTransformName(FlinkBatchTranslationContext context) { return context.getCurrentTransform().getFullName(); } @@ -604,7 +614,7 @@ public void translateNode( throw new RuntimeException(e); } - Map, Coder> outputCoderMap = context.getOutputCoders(); + final Map, Coder> outputCoderMap = context.getOutputCoders(transform); String fullName = getCurrentTransformName(context); if (usesStateOrTimers) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 8e366cf85d42..1af8020cdc19 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -19,9 +19,10 @@ import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.utils.CountingPipelineVisitor; +import org.apache.beam.runners.flink.translation.utils.LookupPipelineVisitor; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -32,7 +33,6 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -57,9 +57,12 @@ class FlinkBatchTranslationContext { private AppliedPTransform currentTransform; + private final CountingPipelineVisitor countingPipelineVisitor = new CountingPipelineVisitor(); + private final LookupPipelineVisitor lookupPipelineVisitor = new LookupPipelineVisitor(); + // ------------------------------------------------------------------------ - public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { + FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { this.env = env; this.options = options; this.dataSets = new HashMap<>(); @@ -68,13 +71,18 @@ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions op this.danglingDataSets = new HashMap<>(); } + void init(Pipeline pipeline) { + pipeline.traverseTopologically(countingPipelineVisitor); + pipeline.traverseTopologically(lookupPipelineVisitor); + } + // ------------------------------------------------------------------------ - public Map> getDanglingDataSets() { + Map> getDanglingDataSets() { return danglingDataSets; } - public ExecutionEnvironment getExecutionEnvironment() { + ExecutionEnvironment getExecutionEnvironment() { return env; } @@ -83,13 +91,13 @@ public PipelineOptions getPipelineOptions() { } @SuppressWarnings("unchecked") - public DataSet> getInputDataSet(PValue value) { + DataSet> getInputDataSet(PValue value) { // assume that the DataSet is used as an input if retrieved here danglingDataSets.remove(value); return (DataSet>) dataSets.get(value); } - public void setOutputDataSet(PValue value, DataSet> set) { + void setOutputDataSet(PValue value, DataSet> set) { if (!dataSets.containsKey(value)) { dataSets.put(value, set); danglingDataSets.put(value, set); @@ -99,41 +107,37 @@ public void setOutputDataSet(PValue value, DataSet> set) { /** * Sets the AppliedPTransform which carries input/output. * - * @param currentTransform + * @param currentTransform Current transformation. */ - public void setCurrentTransform(AppliedPTransform currentTransform) { + void setCurrentTransform(AppliedPTransform currentTransform) { this.currentTransform = currentTransform; } - public AppliedPTransform getCurrentTransform() { + AppliedPTransform getCurrentTransform() { return currentTransform; } - public Map, Coder> getOutputCoders() { - return currentTransform.getOutputs().entrySet().stream() - .filter(e -> e.getValue() instanceof PCollection) - .collect(Collectors.toMap(e -> e.getKey(), e -> ((PCollection) e.getValue()).getCoder())); + Map, Coder> getOutputCoders(PTransform transform) { + return lookupPipelineVisitor.getOutputCoders(transform); } @SuppressWarnings("unchecked") - public DataSet getSideInputDataSet(PCollectionView value) { + DataSet getSideInputDataSet(PCollectionView value) { return (DataSet) broadcastDataSets.get(value); } - public void setSideInputDataSet( + void setSideInputDataSet( PCollectionView value, DataSet> set) { if (!broadcastDataSets.containsKey(value)) { broadcastDataSets.put(value, set); } } - @SuppressWarnings("unchecked") - public TypeInformation> getTypeInfo(PCollection collection) { + TypeInformation> getTypeInfo(PCollection collection) { return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy()); } - @SuppressWarnings("unchecked") - public TypeInformation> getTypeInfo( + TypeInformation> getTypeInfo( Coder coder, WindowingStrategy windowingStrategy) { WindowedValue.FullWindowedValueCoder windowedValueCoder = WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); @@ -142,20 +146,23 @@ public TypeInformation> getTypeInfo( } Map, PValue> getInputs(PTransform transform) { - return currentTransform.getInputs(); + return lookupPipelineVisitor.getInputs(transform); } - @SuppressWarnings("unchecked") T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); + return lookupPipelineVisitor.getInput(transform); } Map, PValue> getOutputs(PTransform transform) { - return currentTransform.getOutputs(); + return lookupPipelineVisitor.getOutputs(transform); } - @SuppressWarnings("unchecked") T getOutput(PTransform transform) { - return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values()); + return lookupPipelineVisitor.getOutput(transform); + } + + /** {@link CountingPipelineVisitor#getNumConsumers(PValue)}. */ + int getNumConsumers(PValue value) { + return countingPipelineVisitor.getNumConsumers(value); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java new file mode 100644 index 000000000000..5f92d9fd652c --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java @@ -0,0 +1,47 @@ +/* + * 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.beam.runners.flink.translation.utils; + +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.values.PValue; + +/** Pipeline visitors that fills a lookup table of {@link PValue} to number of consumers. */ +public class CountingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults { + + private final Map numConsumers = new HashMap<>(); + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + for (PValue input : node.getInputs().values()) { + numConsumers.merge(input, 1, Integer::sum); + } + } + + /** + * Calculate number of consumers of a given {@link PValue}. + * + * @param value PValue to perform calculation for. + * @return Number of consumers. + */ + public int getNumConsumers(PValue value) { + return numConsumers.get(value); + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java new file mode 100644 index 000000000000..662e4d666a6b --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java @@ -0,0 +1,98 @@ +/* + * 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.beam.runners.flink.translation.utils; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.runners.core.construction.TransformInputs; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; + +/** + * Pipeline visitor that fills lookup table of {@link PTransform} to {@link AppliedPTransform} for + * usage in {@link + * org.apache.beam.runners.flink.FlinkBatchPortablePipelineTranslator.BatchTranslationContext}. + */ +public class LookupPipelineVisitor extends Pipeline.PipelineVisitor.Defaults { + + private final Map, AppliedPTransform> lookupTable = new HashMap<>(); + + @Override + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { + if ((node.getTransform()) != null) { + final AppliedPTransform applied = node.toAppliedPTransform(getPipeline()); + lookupTable.put(applied.getTransform(), applied); + } + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + final AppliedPTransform applied = node.toAppliedPTransform(getPipeline()); + lookupTable.put(applied.getTransform(), applied); + } + + private + AppliedPTransform> applied( + PTransform transform) { + @SuppressWarnings("unchecked") + final AppliedPTransform> applied = + (AppliedPTransform>) + lookupTable.get(transform); + if (applied == null) { + throw new IllegalArgumentException( + String.format("AppliedPTransform for %s does not exist.", transform)); + } + return applied; + } + + public Map, PValue> getInputs(PTransform transform) { + return applied(transform).getInputs(); + } + + @SuppressWarnings("unchecked") + public T getInput(PTransform transform) { + return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(applied(transform))); + } + + public Map, PValue> getOutputs(PTransform transform) { + return applied(transform).getOutputs(); + } + + @SuppressWarnings("unchecked") + public T getOutput(PTransform transform) { + return (T) Iterables.getOnlyElement(applied(transform).getOutputs().values()); + } + + @SuppressWarnings("unchecked") + public Map, Coder> getOutputCoders(PTransform transform) { + return getOutputs(transform).entrySet().stream() + .filter(e -> e.getValue() instanceof PCollection) + .collect(Collectors.toMap(Map.Entry::getKey, e -> ((PCollection) e.getValue()).getCoder())); + } +} From 3c4536dc98f4574ab86f873d50796da3de41a804 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Mon, 1 Jun 2020 12:33:24 +0200 Subject: [PATCH 141/151] [BEAM-8848] Flink Batch Runner: optimize GBK translation for non-merging windows. --- .../flink/FlinkBatchTransformTranslators.java | 96 +++++++++++++-- .../runners/flink/FlinkPipelineOptions.java | 7 ++ .../FlinkExplodeWindowsFunction.java | 37 ++++++ .../FlinkNonMergingReduceFunction.java | 112 ++++++++++++++++++ .../types/WindowedKvKeySelector.java | 60 ++++++++++ .../flink/batch/NonMergingGroupByKeyTest.java | 80 +++++++++++++ .../flink_java_pipeline_options.html | 5 + .../flink_python_pipeline_options.html | 5 + 8 files changed, 391 insertions(+), 11 deletions(-) create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java create mode 100644 runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 3f5104d1b7c3..d17b4405dfa2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -34,19 +34,23 @@ import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkExplodeWindowsFunction; import org.apache.beam.runners.flink.translation.functions.FlinkIdentityFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkNonMergingReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction; import org.apache.beam.runners.flink.translation.functions.FlinkStatefulDoFnFunction; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.KvKeySelector; +import org.apache.beam.runners.flink.translation.types.WindowedKvKeySelector; import org.apache.beam.runners.flink.translation.wrappers.ImpulseInputFormat; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -65,6 +69,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -91,6 +96,7 @@ import org.apache.flink.api.java.operators.MapOperator; import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.operators.SingleInputUdfOperator; +import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.Optimizer; import org.joda.time.Instant; @@ -120,18 +126,21 @@ private static void registerTranslator( registerTranslator(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslatorBatch()); registerTranslator( PTransformTranslation.CREATE_VIEW_TRANSFORM_URN, - new CreatePCollectionViewTranslatorBatch()); + new CreatePCollectionViewTranslatorBatch<>()); registerTranslator( - PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, new CombinePerKeyTranslatorBatch()); + PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, new CombinePerKeyTranslatorBatch<>()); registerTranslator( - PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslatorBatch()); - registerTranslator(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch()); + PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, + new NonMergingGroupByKeyTranslatorBatch<>()); registerTranslator( - PTransformTranslation.FLATTEN_TRANSFORM_URN, new FlattenPCollectionTranslatorBatch()); + PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslatorBatch<>()); + registerTranslator(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch<>()); registerTranslator( - PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, new WindowAssignTranslatorBatch()); - registerTranslator(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch()); - registerTranslator(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch()); + PTransformTranslation.FLATTEN_TRANSFORM_URN, new FlattenPCollectionTranslatorBatch<>()); + registerTranslator( + PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, new WindowAssignTranslatorBatch<>()); + registerTranslator(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch<>()); + registerTranslator(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch<>()); } @SuppressWarnings("unchecked") @@ -139,7 +148,7 @@ static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator( PTransform transform, FlinkBatchTranslationContext context) { @Nullable final String urn = PTransformTranslation.urnForTransformOrNull(transform); if (urn != null && TRANSLATORS.containsKey(urn)) { - for (FlinkBatchPipelineTranslator.BatchTransformTranslator translator : + for (FlinkBatchPipelineTranslator.BatchTransformTranslator> translator : TRANSLATORS.get(urn)) { if (translator.canTranslate(transform, context)) { return translator; @@ -241,6 +250,73 @@ public void translateNode( } } + /** + * Optimized group by key translation for non-merging windows. + * + * @param Key type. + * @param Input type. + */ + private static class NonMergingGroupByKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + PTransform>, PCollection>>>> { + + @Override + public boolean canTranslate( + PTransform>, PCollection>>> transform, + FlinkBatchTranslationContext context) { + final WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + return windowingStrategy.getWindowFn().isNonMerging() + && windowingStrategy.getTimestampCombiner() == TimestampCombiner.END_OF_WINDOW + && windowingStrategy.getWindowFn().windowCoder().consistentWithEquals(); + } + + @Override + public void translateNode( + PTransform>, PCollection>>> transform, + FlinkBatchTranslationContext context) { + final int numConsumers = + context.getOutputs(transform).values().stream().mapToInt(context::getNumConsumers).sum(); + final boolean multipleConsumers = numConsumers > 1; + final boolean reIterableResult = + multipleConsumers + || context + .getPipelineOptions() + .as(FlinkPipelineOptions.class) + .getReIterableGroupByKeyResult(); + final DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + final KvCoder inputCoder = + (KvCoder) context.getInput(transform).getCoder(); + final WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + final String fullName = getCurrentTransformName(context); + final UnsortedGrouping>> inputGrouping = + new FlatMapOperator<>( + inputDataSet, + inputDataSet.getType(), + new FlinkExplodeWindowsFunction<>(), + "ExplodeWindows: " + fullName) + .groupBy( + new WindowedKvKeySelector<>( + inputCoder.getKeyCoder(), windowingStrategy.getWindowFn().windowCoder())); + final TypeInformation>>> outputTypeInfo = + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of( + inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())), + windowingStrategy.getWindowFn().windowCoder())); + final DataSet>>> outputDataSet = + new GroupReduceOperator<>( + inputGrouping, + outputTypeInfo, + new FlinkNonMergingReduceFunction<>(windowingStrategy, reIterableResult), + fullName) + .returns(outputTypeInfo); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } + private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator< PTransform>, PCollection>>>> { @@ -806,6 +882,4 @@ private static void transformSideInputs( outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); } } - - private FlinkBatchTransformTranslators() {} } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 22bff18dbdc2..6c42e2f5efb4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -267,4 +267,11 @@ public interface FlinkPipelineOptions String getReportCheckpointDuration(); void setReportCheckpointDuration(String metricNamespace); + + @Description( + "Flag indicating whether result of GBK needs to be re-iterable. Re-iterable result implies that all values for a single key must fit in memory as we currently do not support spilling to disk.") + @Default.Boolean(false) + Boolean getReIterableGroupByKeyResult(); + + void setReIterableGroupByKeyResult(Boolean reIterableGroupByKeyResult); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java new file mode 100644 index 000000000000..b307878fb025 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java @@ -0,0 +1,37 @@ +/* + * 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.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.util.Collector; + +/** + * Explode {@link WindowedValue} that belongs to multiple windows into multiple "single window" + * {@link WindowedValue values}, so we can safely group elements by (K, W) tuples. + * + * @param Value type. + */ +public class FlinkExplodeWindowsFunction + implements FlatMapFunction, WindowedValue> { + + @Override + public void flatMap(WindowedValue value, Collector> coll) { + value.explodeWindows().forEach(coll::collect); + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java new file mode 100644 index 000000000000..8855411472f6 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java @@ -0,0 +1,112 @@ +/* + * 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.beam.runners.flink.translation.functions; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.PeekingIterator; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +/** + * Reduce function for non-merging GBK implementation. Implementation tries to return non-iterable + * results when possible, so we do not have to materialize all values for a single key in memory. + * + * @param Key type. + * @param Input type. + */ +public class FlinkNonMergingReduceFunction + implements GroupReduceFunction< + WindowedValue>, WindowedValue>>> { + + private static class OnceIterable implements Iterable { + + private final Iterator iterator; + + private final AtomicBoolean used = new AtomicBoolean(false); + + OnceIterable(Iterator iterator) { + this.iterator = iterator; + } + + @Override + public Iterator iterator() { + if (used.compareAndSet(false, true)) { + return iterator; + } + throw new IllegalStateException( + "GBK result is not re-iterable. You can enable re-iterations by setting '--reIterableGroupByKeyResult=true'."); + } + } + + private final WindowingStrategy windowingStrategy; + private final boolean reIterableResult; + + public FlinkNonMergingReduceFunction( + WindowingStrategy windowingStrategy, boolean reIterableResult) { + this.windowingStrategy = windowingStrategy; + this.reIterableResult = reIterableResult; + } + + @Override + public void reduce( + Iterable>> input, + Collector>>> coll) { + final PeekingIterator>> iterator = + Iterators.peekingIterator(input.iterator()); + final WindowedValue> first = iterator.peek(); + final BoundedWindow window = Iterables.getOnlyElement(first.getWindows()); + @SuppressWarnings("unchecked") + final Instant outputTimestamp = + ((WindowingStrategy) windowingStrategy) + .getWindowFn() + .getOutputTime(first.getTimestamp(), window); + final Instant combinedTimestamp = + windowingStrategy.getTimestampCombiner().assign(window, outputTimestamp); + final Iterable values; + if (reIterableResult) { + final List lst = new ArrayList<>(); + iterator.forEachRemaining(wv -> lst.add(wv.getValue().getValue())); + values = lst; + } else { + values = + new OnceIterable<>( + Iterators.transform( + iterator, + (WindowedValue> wv) -> + Objects.requireNonNull(wv).getValue().getValue())); + } + coll.collect( + WindowedValue.of( + KV.of(first.getValue().getKey(), values), + combinedTimestamp, + first.getWindows(), + PaneInfo.ON_TIME_AND_ONLY_FIRING)); + } +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java new file mode 100644 index 000000000000..4158c026e203 --- /dev/null +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java @@ -0,0 +1,60 @@ +/* + * 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.beam.runners.flink.translation.types; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Bytes; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; + +/** + * {@link KeySelector} that extracts the key from a {@link KV} and returns it in encoded form as a + * {@code byte} array. + */ +public class WindowedKvKeySelector + implements KeySelector>, byte[]>, ResultTypeQueryable { + + private final Coder keyCoder; + private final Coder windowCoder; + + public WindowedKvKeySelector(Coder keyCoder, Coder windowCoder) { + this.keyCoder = keyCoder; + this.windowCoder = windowCoder; + } + + @Override + public byte[] getKey(WindowedValue> value) throws Exception { + final byte[] encodedKey = CoderUtils.encodeToByteArray(keyCoder, value.getValue().getKey()); + @SuppressWarnings("unchecked") + final byte[] encodedWindow = + CoderUtils.encodeToByteArray( + (Coder) windowCoder, Iterables.getOnlyElement(value.getWindows())); + return Bytes.concat(encodedKey, encodedWindow); + } + + @Override + public TypeInformation getProducedType() { + return new EncodedValueTypeInformation(); + } +} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java new file mode 100644 index 000000000000..935ca21e6bf0 --- /dev/null +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java @@ -0,0 +1,80 @@ +/* + * 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.beam.runners.flink.batch; + +import java.util.Arrays; +import java.util.Objects; +import org.apache.beam.runners.flink.FlinkCapabilities; +import org.apache.beam.runners.flink.FlinkPipelineOptions; +import org.apache.beam.runners.flink.FlinkTestPipeline; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.flink.test.util.AbstractTestBase; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; + +public class NonMergingGroupByKeyTest extends AbstractTestBase { + + private static class ReiterateDoFn extends DoFn>, Void> { + + @ProcessElement + public void processElement(@Element KV> el) { + el.getValue().iterator(); + // this should throw an exception + el.getValue().iterator(); + } + } + + @Test + public void testDisabledReIterationThrowsAnException() { + // If output during closing is not supported, we can not chain DoFns and results + // are therefore materialized during output serialization. + Assume.assumeTrue(FlinkCapabilities.supportsOutputDuringClosing()); + final Pipeline p = FlinkTestPipeline.createForBatch(); + p.apply(Create.of(Arrays.asList(KV.of("a", 1), KV.of("b", 2), KV.of("c", 3)))) + .apply(GroupByKey.create()) + .apply(ParDo.of(new ReiterateDoFn<>())); + Pipeline.PipelineExecutionException resultException = null; + try { + p.run().waitUntilFinish(); + } catch (Pipeline.PipelineExecutionException exception) { + resultException = exception; + } + Assert.assertEquals( + IllegalStateException.class, Objects.requireNonNull(resultException).getCause().getClass()); + Assert.assertTrue( + resultException.getCause().getMessage().contains("GBK result is not re-iterable.")); + } + + @Test + public void testEnabledReIterationDoesNotThrowAnException() { + final Pipeline p = FlinkTestPipeline.createForBatch(); + p.getOptions().as(FlinkPipelineOptions.class).setReIterableGroupByKeyResult(true); + p.apply(Create.of(Arrays.asList(KV.of("a", 1), KV.of("b", 2), KV.of("c", 3)))) + .apply(GroupByKey.create()) + .apply(ParDo.of(new ReiterateDoFn<>())); + final PipelineResult.State state = p.run().waitUntilFinish(); + Assert.assertEquals(PipelineResult.State.DONE, state); + } +} diff --git a/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html b/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html index 317b06141c37..956446920c71 100644 --- a/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html +++ b/website/www/site/layouts/shortcodes/flink_java_pipeline_options.html @@ -132,6 +132,11 @@ The degree of parallelism to be used when distributing operations onto workers. If the parallelism is not set, the configured Flink default is used, or 1 if none can be found. Default: -1 + + reIterableGroupByKeyResult + Flag indicating whether result of GBK needs to be re-iterable. Re-iterable result implies that all values for a single key must fit in memory as we currently do not support spilling to disk. + Default: false + reportCheckpointDuration If not null, reports the checkpoint duration of each ParDo stage in the provided metric namespace. diff --git a/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html b/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html index 4170e25c543c..9abe5086df74 100644 --- a/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html +++ b/website/www/site/layouts/shortcodes/flink_python_pipeline_options.html @@ -132,6 +132,11 @@ The degree of parallelism to be used when distributing operations onto workers. If the parallelism is not set, the configured Flink default is used, or 1 if none can be found. Default: -1 + + re_iterable_group_by_key_result + Flag indicating whether result of GBK needs to be re-iterable. Re-iterable result implies that all values for a single key must fit in memory as we currently do not support spilling to disk. + Default: false + report_checkpoint_duration If not null, reports the checkpoint duration of each ParDo stage in the provided metric namespace. From 23a1fb711bce2af3a42b06f522139babcc72b285 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Thu, 26 Mar 2020 10:03:19 +0100 Subject: [PATCH 142/151] [BEAM-8848] Code review. --- .../flink/FlinkBatchTransformTranslators.java | 34 ++++++++----------- .../FlinkNonMergingReduceFunction.java | 2 +- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index d17b4405dfa2..264fa4c43302 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -83,6 +83,8 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.MultimapBuilder; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -111,36 +113,28 @@ class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map> - TRANSLATORS = new HashMap<>(); - - private static void registerTranslator( - String urn, FlinkBatchPipelineTranslator.BatchTransformTranslator translator) { - if (!TRANSLATORS.containsKey(urn)) { - TRANSLATORS.put(urn, new ArrayList<>()); - } - TRANSLATORS.get(urn).add(translator); - } + private static final Multimap + TRANSLATORS = MultimapBuilder.hashKeys().arrayListValues().build(); static { - registerTranslator(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslatorBatch()); - registerTranslator( + TRANSLATORS.put(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslatorBatch()); + TRANSLATORS.put( PTransformTranslation.CREATE_VIEW_TRANSFORM_URN, new CreatePCollectionViewTranslatorBatch<>()); - registerTranslator( + TRANSLATORS.put( PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, new CombinePerKeyTranslatorBatch<>()); - registerTranslator( + TRANSLATORS.put( PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new NonMergingGroupByKeyTranslatorBatch<>()); - registerTranslator( + TRANSLATORS.put( PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslatorBatch<>()); - registerTranslator(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch<>()); - registerTranslator( + TRANSLATORS.put(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslatorBatch<>()); + TRANSLATORS.put( PTransformTranslation.FLATTEN_TRANSFORM_URN, new FlattenPCollectionTranslatorBatch<>()); - registerTranslator( + TRANSLATORS.put( PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, new WindowAssignTranslatorBatch<>()); - registerTranslator(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch<>()); - registerTranslator(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch<>()); + TRANSLATORS.put(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoTranslatorBatch<>()); + TRANSLATORS.put(PTransformTranslation.READ_TRANSFORM_URN, new ReadSourceTranslatorBatch<>()); } @SuppressWarnings("unchecked") diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java index 8855411472f6..21a9cac71283 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java @@ -61,7 +61,7 @@ public Iterator iterator() { return iterator; } throw new IllegalStateException( - "GBK result is not re-iterable. You can enable re-iterations by setting '--reIterableGroupByKeyResult=true'."); + "GBK result is not re-iterable. You can enable re-iterations by setting '--reIterableGroupByKeyResult'."); } } From 423b79ce28de9f0876859b1fae03a74d101d8ccf Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 10:59:33 -0700 Subject: [PATCH 143/151] wip --- .../sdk/transforms/SerializableFunction.java | 1 - .../beam/sdk/io/gcp/healthcare/FhirIO.java | 61 +++---- .../healthcare/HttpHealthcareApiClient.java | 15 +- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 44 ++--- .../sdk/io/gcp/healthcare/FhirIOUpdateIT.java | 158 ++++++++++++++++++ .../sdk/io/gcp/healthcare/FhirIOWriteIT.java | 52 ------ 6 files changed, 221 insertions(+), 110 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java index cbd7d01d80fe..f1e870513f10 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import com.fasterxml.jackson.core.JsonProcessingException; import java.io.IOException; import java.io.Serializable; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 19ece0754b9d..911db6c8bc3c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -1284,7 +1284,7 @@ public void executeBundles(ProcessContext context) { * @param fhirStore the fhir store * @return the fhir io . create resources */ - public static FhirIO.CreateResources createResources(ValueProvider fhirStore){ + public static FhirIO.CreateResources createResources(ValueProvider fhirStore) { return new CreateResources(fhirStore); } @@ -1295,7 +1295,7 @@ public static FhirIO.CreateResources createResources(ValueProvider FhirIO.CreateResources createResources(String fhirStore){ + public static FhirIO.CreateResources createResources(String fhirStore) { return new CreateResources(fhirStore); } /** @@ -1353,8 +1353,7 @@ public CreateResources withIfNotExistFunction( * @param typeFunction for extracting type from a resource. * @return the create resources */ - public CreateResources withTypeFunction( - SerializableFunction typeFunction) { + public CreateResources withTypeFunction(SerializableFunction typeFunction) { this.typeFunction = typeFunction; return this; } @@ -1388,7 +1387,8 @@ public FhirIO.Write.Result expand(PCollection input) { input .apply( ParDo.of( - new CreateFn(fhirStore, typeFunction, formatBodyFunction, ifNoneExistFunction))) + new CreateFn( + fhirStore, typeFunction, formatBodyFunction, ifNoneExistFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1419,8 +1419,7 @@ public void initClient() throws IOException { } @ProcessElement - public void create(ProcessContext context) - throws IOException { + public void create(ProcessContext context) throws IOException { T input = context.element(); String body = formatBodyFunction.apply(input); String type = typeFunction.apply(input); @@ -1448,7 +1447,7 @@ public void create(ProcessContext context) * @param fhirStore the fhir store * @return the update resources */ - public static UpdateResources update(ValueProvider fhirStore){ + public static UpdateResources update(ValueProvider fhirStore) { return new UpdateResources(fhirStore); } @@ -1459,24 +1458,24 @@ public static UpdateResources update(ValueProvider fhi * @param fhirStore the fhir store * @return the update resources */ - public static UpdateResources update(String fhirStore){ + public static UpdateResources update(String fhirStore) { return new UpdateResources(fhirStore); } /** * {@link PTransform} for Updating FHIR resources resources. * - * This transform assumes the input {@link PCollection} contains - * {@link KV} of resource name, value pairs and by default will call {@code .toString} to extract - * string values. - * However, the user can override this behavior by specifying a {@link SerializableFunction} with - * custom logic to extract the resource name and body from the {@link KV} in - * {@link UpdateResources#withResourceNameFunction(SerializableFunction)} and - * {@link UpdateResources#withFormatBodyFunction(SerializableFunction)} + *

    This transform assumes the input {@link PCollection} contains {@link KV} of resource name, + * value pairs and by default will call {@code .toString} to extract string values. However, the + * user can override this behavior by specifying a {@link SerializableFunction} with custom logic + * to extract the resource name and body from the {@link KV} in {@link + * UpdateResources#withResourceNameFunction(SerializableFunction)} and {@link + * UpdateResources#withFormatBodyFunction(SerializableFunction)} * *

    https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/update */ - public static class UpdateResources extends PTransform, Write.Result> { + public static class UpdateResources + extends PTransform, Write.Result> { private final ValueProvider fhirStore; private SerializableFunction formatBodyFunction; private SerializableFunction resourceNameFunction; @@ -1530,13 +1529,11 @@ public UpdateResources withFormatBodyFunction( * @param etagFunction ETag function * @return the update resources */ - public UpdateResources withETagFunction( - SerializableFunction etagFunction) { + public UpdateResources withETagFunction(SerializableFunction etagFunction) { this.etagFunction = etagFunction; return this; } - @Override public FhirIO.Write.Result expand(PCollection input) { checkArgument( @@ -1548,7 +1545,10 @@ public FhirIO.Write.Result expand(PCollection input) { return Write.Result.in( input.getPipeline(), input - .apply(ParDo.of(new UpdateFn(fhirStore, formatBodyFunction, resourceNameFunction, etagFunction))) + .apply( + ParDo.of( + new UpdateFn( + fhirStore, formatBodyFunction, resourceNameFunction, etagFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1579,8 +1579,7 @@ public void initClient() throws IOException { } @ProcessElement - public void update(ProcessContext context) - throws IOException { + public void update(ProcessContext context) throws IOException { T input = context.element(); String body = formatBodyFunction.apply(input); try { @@ -1588,7 +1587,7 @@ public void update(ProcessContext context) mapper.readTree(body); String resourceName = resourceNameFunction.apply(input); String etag = null; - if (etagFunction != null){ + if (etagFunction != null) { etag = etagFunction.apply(input); } HttpBody result = client.fhirUpdate(fhirStore.get(), resourceName, body, etag); @@ -1608,8 +1607,8 @@ public void update(ProcessContext context) * @param fhirStore the fhir store * @return the conditional update */ - public static ConditionalUpdate conditionalUpdate(ValueProvider fhirStore){ - return new ConditionalUpdate(fhirStore); + public static ConditionalUpdate conditionalUpdate(ValueProvider fhirStore) { + return new ConditionalUpdate(fhirStore); } /** @@ -1619,7 +1618,7 @@ public static ConditionalUpdate conditionalUpdate(ValueProvider f * @param fhirStore the fhir store * @return the conditional update */ - public static ConditionalUpdate conditionalUpdate(String fhirStore){ + public static ConditionalUpdate conditionalUpdate(String fhirStore) { return new ConditionalUpdate(fhirStore); } @@ -1653,7 +1652,6 @@ public static class ConditionalUpdate extends PTransform, Writ this.fhirStore = StaticValueProvider.of(fhirStore); } - /** * With search parameters function conditional update. * @@ -1714,7 +1712,10 @@ public FhirIO.Write.Result expand(PCollection input) { .apply( ParDo.of( new ConditionalUpdateFn( - fhirStore, typeFunction, searchParametersFunction, formatBodyFunction, + fhirStore, + typeFunction, + searchParametersFunction, + formatBodyFunction, etagFunction))) .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); } @@ -1759,7 +1760,7 @@ public void conditionalUpdate(ProcessContext context) throws IOException { mapper.readTree(body); Map searchParameters = searchParametersFunction.apply(input); String etag = null; - if (etagFunction != null){ + if (etagFunction != null) { etag = etagFunction.apply(input); } client.fhirConditionalUpdate(fhirStore.get(), type, body, searchParameters, etag); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index dbad6125dbc1..f0c02f4a08fb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -460,10 +460,7 @@ public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) if (!Strings.isNullOrEmpty(fhirHttpRequest.pathSuffix)) { uriString += fhirHttpRequest.pathSuffix; } - uri = - new URIBuilder(uriString) - .setParameter("access_token", credentials.getAccessToken().getTokenValue()) - .build(); + uri = new URIBuilder(uriString).build(); } catch (URISyntaxException e) { LOG.error("URL error when making executeBundle request to FHIR API. " + e.getMessage()); throw new IllegalArgumentException(e); @@ -483,6 +480,7 @@ public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) requestBuilder .setUri(uri) .setEntity(requestEntity) + .addHeader("Authorization", "Bearer " + credentials.getAccessToken().getTokenValue()) .addHeader("User-Agent", USER_AGENT) .addHeader("Content-Type", FHIRSTORE_HEADER_CONTENT_TYPE) .addHeader("Accept-Charset", FHIRSTORE_HEADER_ACCEPT_CHARSET) @@ -532,7 +530,10 @@ public HttpBody fhirCreate( @Override public HttpBody fhirConditionalUpdate( - String fhirStore, String type, String resource, Map searchParameters, + String fhirStore, + String type, + String resource, + Map searchParameters, @Nullable String etag) throws IOException, HealthcareHttpException { Map headers = new HashMap<>(); @@ -548,8 +549,8 @@ public HttpBody fhirConditionalUpdate( } @Override - public HttpBody fhirUpdate(String fhirStore, String relativeResourceName, String resource, - @Nullable String etag) + public HttpBody fhirUpdate( + String fhirStore, String relativeResourceName, String resource, @Nullable String etag) throws IOException, HealthcareHttpException { Map headers = new HashMap<>(); if (etag != null) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 61af5fa8f50b..385d6403297a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -45,37 +45,42 @@ class FhirIOTestUtil { public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests"; - - public static class ExtractIDSearchQuery implements - SerializableFunction { + public static class ExtractIDSearchQuery implements SerializableFunction { private ObjectMapper mapper; - ExtractIDSearchQuery(){ + ExtractIDSearchQuery() { mapper = new ObjectMapper(); } @Override - public String apply(String resource) throws IOException { - Map map = mapper.readValue(resource.toString().getBytes(), Map.class); - String id = map.get("id"); - return String.format("_id=%s", id); - } + public String apply(String resource) { + try { + Map map = mapper.readValue(resource.getBytes(), Map.class); + String id = map.get("id"); + return String.format("_id=%s", id); + } catch (IOException e) { + throw new RuntimeException(e); + } + } } - public static class GetByKey implements - SerializableFunction { + public static class GetByKey implements SerializableFunction { private final String key; private ObjectMapper mapper; - public GetByKey(String key){ + public GetByKey(String key) { this.key = key; mapper = new ObjectMapper(); } @Override - public String apply(String resource) throws IOException { - Map map = mapper.readValue(resource.getBytes(), Map.class); - return map.get(key); + public String apply(String resource) { + try { + Map map = mapper.readValue(resource.getBytes(), Map.class); + return map.get(key); + } catch (IOException e) { + throw new RuntimeException(e); + } } } @@ -83,9 +88,8 @@ public String apply(String resource) throws IOException { // TODO read update resources function. // TODO spot check resource update utility. - private static Stream readAllTestResources(String subDir, String version ) { - Path resourceDir = Paths.get("build", "resources", "test", - subDir, version); + private static Stream readAllTestResources(String subDir, String version) { + Path resourceDir = Paths.get("build", "resources", "test", subDir, version); String absolutePath = resourceDir.toFile().getAbsolutePath(); File dir = new File(absolutePath); File[] fhirJsons = dir.listFiles(); @@ -100,8 +104,8 @@ private static Stream readAllTestResources(String subDir, String version } }) .map(String::new); - } + private static Stream readPrettyBundles(String version) { return readAllTestResources("transactional_bundles", version); } @@ -134,7 +138,7 @@ private static Stream readPrettyResources(String version) { readPrettyResources("DSTU2").collect(Collectors.toList()); static final List STU3_PRETTY_RESOURCES = readPrettyResources("STU3").collect(Collectors.toList()); - static final List R4_PRETTY_RESOURCES= + static final List R4_PRETTY_RESOURCES = readPrettyResources("R4").collect(Collectors.toList()); static final Map> RESOURCES; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java new file mode 100644 index 000000000000..2a03a3448445 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java @@ -0,0 +1,158 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.BUNDLES; +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.RESOURCES; +import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; + +import java.io.IOException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.KV; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class FhirIOUpdateIT { + + @Parameters(name = "{0}") + public static Collection versions() { + return Arrays.asList("DSTU2", "STU3", "R4"); + } + + private final String fhirStoreName; + private FhirIOTestOptions options; + private transient HealthcareApiClient client; + private String healthcareDataset; + private long testTime = System.currentTimeMillis(); + + public String version; + + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + + public FhirIOUpdateIT(String version) { + this.version = version; + this.fhirStoreName = + "FHIR_store_" + version + "_write_it_" + testTime + "_" + (new SecureRandom().nextInt(32)); + } + + @Before + public void setup() throws Exception { + if (client == null) { + client = new HttpHealthcareApiClient(); + } + PipelineOptionsFactory.register(FhirIOTestOptions.class); + String project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + healthcareDataset = String.format(HEALTHCARE_DATASET_TEMPLATE, project); + options = TestPipeline.testingPipelineOptions().as(FhirIOTestOptions.class); + options.setGcsTempPath( + String.format("gs://%s/FhirIOWrite%sIT/%s/temp/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setGcsDeadLetterPath( + String.format( + "gs://%s/FhirIOWrite%sIT/%s/deadletter/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.createFhirStore(healthcareDataset, fhirStoreName, version); + } + + @After + public void teardownFhirStore() throws IOException { + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.deleteFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + // clean up GCS objects if any. + } + + @AfterClass + public static void teardownBucket() throws IOException { + FhirIOTestUtil.tearDownTempBucket(); + } + + // TODO(jaketf) add IT for conditional create, update, conditional update transforms. + @Test + public void testFhirIO_CreateResources() { + Result writeResult = + (Result) + pipeline + .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply( + FhirIO.createResources(options.getFhirStore()) + .withTypeFunction(new GetByKey("resourceType")) + .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) + .withFormatBodyFunction(x -> x)); + + PAssert.that(writeResult.getFailedBodies()).empty(); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void testFhirIO_Update() { + // TODO write initial resources to FHIR + Result writeResult = + (Result) + pipeline + .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) + .apply( + "Update Resources", + FhirIO.>update(options.getFhirStore()) + .withResourceNameFunction(x -> x.getKey()) + .withFormatBodyFunction(x -> ((KV) x).getValue())); + + PAssert.that(writeResult.getFailedBodies()).empty(); + + pipeline.run().waitUntilFinish(); + // TODO spot check update results + } + + @Test + public void testFhirIO_ConditionalUpdate() { + // TODO write initial resources to FHIR + Result writeResult = + (Result) + pipeline + .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply( + "Conditional Update Resources", + FhirIO.conditionalUpdate(options.getFhirStore()) + .withTypeFunction(x -> "patient") + .withFormatBodyFunction(x -> "{}") + .withSearchParametersFunction(x -> new HashMap<>())); + // TODO spot check update results + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java index 4f127c050add..cef78fdb70bb 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java @@ -26,9 +26,7 @@ import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; -import com.fasterxml.jackson.databind.ObjectMapper; import java.util.HashMap; -import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; @@ -134,54 +132,4 @@ public void testFhirIO_Import() { pipeline.run().waitUntilFinish(); } - - // TODO(jaketf) add IT for conditional create, update, conditional update transforms. - @Test - public void testFhirIO_CreateResources() { - FhirIO.Write.Result writeResult = - (Result) pipeline - .apply(Create.of(BUNDLES.get(version))) - .apply(FhirIO.createResources(options.getFhirStore()) - .withTypeFunction(new GetByKey("resourceType")) - .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) - .withFormatBodyFunction(x -> x) - ); - - - PAssert.that(writeResult.getFailedBodies()).empty(); - - pipeline.run().waitUntilFinish(); - } - - @Test - public void testFhirIO_Update() { - // TODO write initial resources to FHIR - FhirIO.Write.Result writeResult = - (Result) pipeline - .apply("Create Test Resources", Create.of(RESOURCES.get(version))) - .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) - .apply("Update Resources", - FhirIO.>update(options.getFhirStore()) - .withResourceNameFunction(x -> x.getKey()) - .withFormatBodyFunction(x -> ((KV) x).getValue())); - - PAssert.that(writeResult.getFailedBodies()).empty(); - - pipeline.run().waitUntilFinish(); - // TODO spot check update results - } - - @Test - public void testFhirIO_ConditionalUpdate() { - // TODO write initial resources to FHIR - FhirIO.Write.Result writeResult = - (Result) pipeline - .apply("Create Test Resources", Create.of(RESOURCES.get(version))) - .apply("Conditional Update Resources", - FhirIO.conditionalUpdate(options.getFhirStore()) - .withTypeFunction(x -> "patient") - .withFormatBodyFunction(x -> "{}") - .withSearchParametersFunction(x -> new HashMap<>())); - // TODO spot check update results - } } From 6b61b59859a52df6602523e1dd76ab1e6178c0b2 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 16:32:27 -0700 Subject: [PATCH 144/151] create IT --- .../sdk/transforms/SerializableFunction.java | 3 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 1 - .../healthcare/HttpHealthcareApiClient.java | 3 +- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 44 +- .../sdk/io/gcp/healthcare/FhirIOUpdateIT.java | 19 +- ..._ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json | 14018 --- ..._1fd2683f-2a56-47c7-b674-98f2cc8319e7.json | 10877 --- ..._4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json | 17399 ---- ..._8ed31d3e-7352-4055-89c4-f017db3f594e.json | 9279 -- ..._f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json | 10047 -- .../hospitalInformation1586309771387.json | 825 - .../practitionerInformation1586309771387.json | 617 - ..._d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json | 16391 ---- ..._af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json | 31841 ------ ..._a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json | 16971 ---- ..._fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json | 80348 ---------------- ..._55a5307d-5f23-49c1-9100-7d5c513abca3.json | 16018 --- .../R4/hospitalInformation1586368892823.json | 1182 - .../practitionerInformation1586368892823.json | 978 - ..._02a63c07-9fcc-42ba-aec0-9d5399ac4796.json | 16003 --- ..._88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json | 15372 --- ..._df27a976-5c5e-4b84-ad00-fe32972dce9c.json | 22688 ----- ..._136f997d-0a94-4573-97da-a53b5060a612.json | 28126 ------ ..._74779846-85a4-4b26-9da4-414a5fec1aed.json | 18231 ---- ..._3024090f-fe14-40a9-8fae-79952d3c95ce.json | 32061 ------ .../hospitalInformation1586298239556.json | 880 - .../practitionerInformation1586298239556.json | 523 - .../hospitalInformation1589831190233.json | 183 - .../practitionerInformation1589831190233.json | 135 - .../R4/hospitalInformation1589831190233.json | 318 - .../practitionerInformation1589831190233.json | 354 - .../hospitalInformation1589831190233.json | 234 - .../practitionerInformation1589831190233.json | 135 - 33 files changed, 33 insertions(+), 362071 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json delete mode 100644 sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java index f1e870513f10..a1dba9e688fe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunction.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import java.io.IOException; import java.io.Serializable; /** @@ -36,5 +35,5 @@ public interface SerializableFunction extends ProcessFunction, Serializable { /** Returns the result of invoking this function on the given input. */ @Override - OutputT apply(InputT input) throws IOException; + OutputT apply(InputT input); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 160c60af2cf3..c74ec8e3e072 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -132,7 +132,6 @@ * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link * HealthcareIOError} containing the {@link String} that failed to be ingested and the * exception. - * *

    Conditional Creating / Updating Resources

    * {@link FhirIO} supports interfaces for conditional update. These can be useful to handle * scenarios where an executeBundle failed. For example if you tried to create a resource that diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index a6d2bc0566c7..40b368d2a920 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -417,6 +417,7 @@ enum Method { this.payload = payload; this.method = Method.POST; this.headers = new HashMap<>(); + this.parameters = new HashMap<>(); } public static FhirHttpRequest of(String fhirStore, String payload) { @@ -486,7 +487,7 @@ public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) .addHeader("Accept-Charset", FHIRSTORE_HEADER_ACCEPT_CHARSET) .addHeader("Accept", FHIRSTORE_HEADER_ACCEPT); - // add additional headers + // add additional parameters for (Map.Entry param : fhirHttpRequest.parameters.entrySet()) { requestBuilder.addParameter(param.getKey(), param.getValue()); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 385d6403297a..0de5d9062d70 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -55,13 +56,14 @@ public static class ExtractIDSearchQuery implements SerializableFunction map = mapper.readValue(resource.getBytes(), Map.class); + Map map = + mapper.readValue(resource.getBytes(StandardCharsets.UTF_8), Map.class); String id = map.get("id"); return String.format("_id=%s", id); } catch (IOException e) { throw new RuntimeException(e); } - } + } } public static class GetByKey implements SerializableFunction { @@ -76,7 +78,8 @@ public GetByKey(String key) { @Override public String apply(String resource) { try { - Map map = mapper.readValue(resource.getBytes(), Map.class); + Map map = + mapper.readValue(resource.getBytes(StandardCharsets.UTF_8), Map.class); return map.get(key); } catch (IOException e) { throw new RuntimeException(e); @@ -106,21 +109,21 @@ private static Stream readAllTestResources(String subDir, String version .map(String::new); } - private static Stream readPrettyBundles(String version) { - return readAllTestResources("transactional_bundles", version); + private static List readPrettyBundles(String version) { + return readAllTestResources("transactional_bundles", version).collect(Collectors.toList()); } - private static Stream readPrettyResources(String version) { - return readAllTestResources("resources", version); + private static List readResources(String version) { + return readAllTestResources("resources", version) // stream of file contents + .map((String x) -> x.split("\\r?\\n")) // split lines + .flatMap(Arrays::stream) // flatten lines for all files + .collect(Collectors.toList()); } // Could generate more messages at scale using a tool like // https://synthetichealth.github.io/synthea/ if necessary chose not to avoid the dependency. - static final List DSTU2_PRETTY_BUNDLES = - readPrettyBundles("DSTU2").collect(Collectors.toList()); - static final List STU3_PRETTY_BUNDLES = - readPrettyBundles("STU3").collect(Collectors.toList()); - static final List R4_PRETTY_BUNDLES = - readPrettyBundles("R4").collect(Collectors.toList()); + static final List DSTU2_PRETTY_BUNDLES = readPrettyBundles("DSTU2"); + static final List STU3_PRETTY_BUNDLES = readPrettyBundles("STU3"); + static final List R4_PRETTY_BUNDLES = readPrettyBundles("R4"); static final Map> BUNDLES; @@ -134,20 +137,17 @@ private static Stream readPrettyResources(String version) { // Could generate more messages at scale using a tool like // https://synthetichealth.github.io/synthea/ if necessary chose not to avoid the dependency. - static final List DSTU2_PRETTY_RESOURCES = - readPrettyResources("DSTU2").collect(Collectors.toList()); - static final List STU3_PRETTY_RESOURCES = - readPrettyResources("STU3").collect(Collectors.toList()); - static final List R4_PRETTY_RESOURCES = - readPrettyResources("R4").collect(Collectors.toList()); + static final List DSTU2_RESOURCES = readResources("DSTU2"); + static final List STU3_RESOURCES = readResources("STU3"); + static final List R4_RESOURCES = readResources("R4"); static final Map> RESOURCES; static { Map> m = new HashMap<>(); - m.put("DSTU2", DSTU2_PRETTY_RESOURCES); - m.put("STU3", STU3_PRETTY_RESOURCES); - m.put("R4", R4_PRETTY_RESOURCES); + m.put("DSTU2", DSTU2_RESOURCES); + m.put("STU3", STU3_RESOURCES); + m.put("R4", R4_RESOURCES); RESOURCES = Collections.unmodifiableMap(m); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java index 2a03a3448445..fe5bad28caf4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.healthcare; -import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.BUNDLES; import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.RESOURCES; import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; @@ -27,9 +26,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Import.ContentStructure; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -102,20 +99,20 @@ public static void teardownBucket() throws IOException { FhirIOTestUtil.tearDownTempBucket(); } - // TODO(jaketf) add IT for conditional create, update, conditional update transforms. @Test public void testFhirIO_CreateResources() { - Result writeResult = + Result createResult = (Result) pipeline - .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply("Seed Test Resources", Create.of(RESOURCES.get(version))) .apply( + "Create FHIR Resources", FhirIO.createResources(options.getFhirStore()) .withTypeFunction(new GetByKey("resourceType")) .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) .withFormatBodyFunction(x -> x)); - PAssert.that(writeResult.getFailedBodies()).empty(); + PAssert.that(createResult.getFailedBodies()).empty(); pipeline.run().waitUntilFinish(); } @@ -123,10 +120,10 @@ public void testFhirIO_CreateResources() { @Test public void testFhirIO_Update() { // TODO write initial resources to FHIR - Result writeResult = + Result updateResult = (Result) pipeline - .apply("Create Test Resources", Create.of(RESOURCES.get(version))) + .apply("Seed Test Resources", Create.of(RESOURCES.get(version))) .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) .apply( "Update Resources", @@ -134,7 +131,7 @@ public void testFhirIO_Update() { .withResourceNameFunction(x -> x.getKey()) .withFormatBodyFunction(x -> ((KV) x).getValue())); - PAssert.that(writeResult.getFailedBodies()).empty(); + PAssert.that(updateResult.getFailedBodies()).empty(); pipeline.run().waitUntilFinish(); // TODO spot check update results @@ -143,7 +140,7 @@ public void testFhirIO_Update() { @Test public void testFhirIO_ConditionalUpdate() { // TODO write initial resources to FHIR - Result writeResult = + Result conditionalUpdateResult = (Result) pipeline .apply("Create Test Resources", Create.of(RESOURCES.get(version))) diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json deleted file mode 100644 index 677f4a31ef9e..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Amelia635_Krajcik437_ed7a9f5c-37ea-4767-95ba-a6783e9500b3.json +++ /dev/null @@ -1,14018 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3", - "resource": { - "resourceType": "Patient", - "id": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 3529060201889641636 Population seed: 1586309754086
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Race", - "code": "2106-3", - "display": "White" - } - ], - "text": "White" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Ethnicity", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - ], - "text": "Not Hispanic or Latino" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Shantay950 Collier206" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Quincy", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 2.756674276028057 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 63.24332572397194 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR" - } - ] - }, - "system": "http://hospital.smarthealthit.org", - "value": "ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB" - } - ] - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-91-5467" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "DL" - } - ] - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99971576" - } - ], - "name": [ - { - "use": "official", - "family": [ - "Krajcik437" - ], - "given": [ - "Amelia635" - ], - "prefix": [ - "Mrs." - ] - }, - { - "use": "maiden", - "family": [ - "Hoeger474" - ], - "given": [ - "Amelia635" - ], - "prefix": [ - "Mrs." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-617-5450", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1953-09-25", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 41.56777541317302 - }, - { - "url": "longitude", - "valueDecimal": -70.99411465843632 - } - ] - } - ], - "line": [ - "1056 Stark Trace Suite 42" - ], - "city": "Dartmouth", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "M" - } - ] - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c", - "resource": { - "resourceType": "Organization", - "id": "f9914571-32b8-36d3-a422-4cdce9fbb86c", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "INDEPENDENCE EYE ASSOCIATES, PC", - "telecom": [ - { - "system": "phone", - "value": "508-985-6600" - } - ], - "address": [ - { - "line": [ - "365 FAUNCE CORNER RD" - ], - "city": "DARTMOUTH", - "state": "MA", - "postalCode": "02747-6230", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b", - "resource": { - "resourceType": "Practitioner", - "id": "7e727355-643f-3e2b-b89f-f5076660354b", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "61860" - } - ], - "active": true, - "name": { - "family": [ - "Salazar800" - ], - "given": [ - "Rodrigo242" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "365 FAUNCE CORNER RD" - ], - "city": "DARTMOUTH", - "state": "MA", - "postalCode": "02747-6230", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55", - "resource": { - "resourceType": "Encounter", - "id": "0f99c02d-1199-4d87-8459-123025910c55", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "1993-10-01T19:52:39-07:00", - "end": "1993-10-01T20:07:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3eaf24a3-0c67-4505-a02e-fcc09a4093a3", - "resource": { - "resourceType": "Condition", - "id": "3eaf24a3-0c67-4505-a02e-fcc09a4093a3", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55" - }, - "dateRecorded": "1993-10-01", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "1993-10-01T19:52:39-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:0319b3fc-3373-42f6-9b08-6ee695e640b0", - "resource": { - "resourceType": "CarePlan", - "id": "0319b3fc-3373-42f6-9b08-6ee695e640b0", - "text": { - "status": "generated", - "div": "
    Diabetes self management plan
    " - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "active", - "context": { - "reference": "urn:uuid:0f99c02d-1199-4d87-8459-123025910c55" - }, - "period": { - "start": "1993-10-01T19:52:39-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698360004", - "display": "Diabetes self management plan" - } - ], - "text": "Diabetes self management plan" - } - ], - "addresses": [ - { - "reference": "urn:uuid:3eaf24a3-0c67-4505-a02e-fcc09a4093a3" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "160670007", - "display": "Diabetic diet" - } - ], - "text": "Diabetic diet" - }, - "status": "in-progress", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:fa8baaad-a3ec-4e4a-9cd3-483bb9283cdb", - "resource": { - "resourceType": "Claim", - "id": "fa8baaad-a3ec-4e4a-9cd3-483bb9283cdb", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92", - "resource": { - "resourceType": "Organization", - "id": "ecc51621-0af3-3b35-ac3e-8b1e34022e92", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "SAINT ANNE'S HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5086745600" - } - ], - "address": [ - { - "line": [ - "795 MIDDLE STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02721", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d", - "resource": { - "resourceType": "Practitioner", - "id": "38a05f68-cc3b-33b6-993e-1d2bb2696c3d", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "100" - } - ], - "active": true, - "name": { - "family": [ - "Padberg411" - ], - "given": [ - "Ligia986" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "795 MIDDLE STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02721", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:29a7628c-9d6d-400a-a495-c6714208d248", - "resource": { - "resourceType": "Encounter", - "id": "29a7628c-9d6d-400a-a495-c6714208d248", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "1997-11-21T18:52:39-08:00", - "end": "1997-11-21T19:37:39-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7e78b2fa-5f76-4547-b107-93e9bff9a3fd", - "resource": { - "resourceType": "Condition", - "id": "7e78b2fa-5f76-4547-b107-93e9bff9a3fd", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:29a7628c-9d6d-400a-a495-c6714208d248" - }, - "dateRecorded": "1997-11-21", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - ], - "text": "Miscarriage in first trimester" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "1997-11-21T18:52:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f56d5dc9-3ad3-4b81-9dd3-6b9adcdafeb0", - "resource": { - "resourceType": "Claim", - "id": "f56d5dc9-3ad3-4b81-9dd3-6b9adcdafeb0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315", - "resource": { - "resourceType": "Encounter", - "id": "ed060d5a-328c-4c0f-a88e-8bfb00655315", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2008-09-11T19:52:39-07:00", - "end": "2008-09-11T20:07:39-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "239872002", - "display": "Osteoarthritis of hip" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394", - "resource": { - "resourceType": "Condition", - "id": "b5bfd448-96ad-4bdd-b3be-816b0d018394", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" - }, - "dateRecorded": "2008-09-11", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "239872002", - "display": "Osteoarthritis of hip" - } - ], - "text": "Osteoarthritis of hip" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2008-09-11T19:52:39-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:46597927-f43d-43c3-9482-50dcc695c22a", - "resource": { - "resourceType": "MedicationOrder", - "id": "46597927-f43d-43c3-9482-50dcc695c22a", - "dateWritten": "2008-09-11T19:52:39-07:00", - "status": "active", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "prescriber": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - }, - "encounter": { - "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" - }, - "reasonReference": { - "reference": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "849574", - "display": "Naproxen sodium 220 MG Oral Tablet" - } - ], - "text": "Naproxen sodium 220 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:a237e458-a9ef-4914-88c4-7faed80b0730", - "resource": { - "resourceType": "Claim", - "id": "a237e458-a9ef-4914-88c4-7faed80b0730", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:46597927-f43d-43c3-9482-50dcc695c22a" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6fbc1d80-0ee3-48ae-8efc-e404a9720e5e", - "resource": { - "resourceType": "CarePlan", - "id": "6fbc1d80-0ee3-48ae-8efc-e404a9720e5e", - "text": { - "status": "generated", - "div": "
    Musculoskeletal care
    " - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "active", - "context": { - "reference": "urn:uuid:ed060d5a-328c-4c0f-a88e-8bfb00655315" - }, - "period": { - "start": "2008-09-11T19:52:39-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "408869004", - "display": "Musculoskeletal care" - } - ], - "text": "Musculoskeletal care" - } - ], - "addresses": [ - { - "reference": "urn:uuid:b5bfd448-96ad-4bdd-b3be-816b0d018394" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "386294003", - "display": "Joint mobility exercises" - } - ], - "text": "Joint mobility exercises" - }, - "status": "in-progress", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266694003", - "display": "Heat therapy" - } - ], - "text": "Heat therapy" - }, - "status": "in-progress", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:04cac450-c4b6-41ad-8b78-e3dff5734386", - "resource": { - "resourceType": "Claim", - "id": "04cac450-c4b6-41ad-8b78-e3dff5734386", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "239872002", - "display": "Osteoarthritis of hip" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d", - "resource": { - "resourceType": "Organization", - "id": "f7a6cc79-3075-3124-9f62-6fe562440b1d", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f7a6cc79-3075-3124-9f62-6fe562440b1d" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "HAWTHORN MEDICAL URGENT CARE CENTER", - "telecom": [ - { - "system": "phone", - "value": "508-961-0861" - } - ], - "address": [ - { - "line": [ - "237 STATE ROAD" - ], - "city": "NORTH DARTMOUTH", - "state": "MA", - "postalCode": "2747", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a", - "resource": { - "resourceType": "Practitioner", - "id": "d4b02c0c-6cc4-31dd-bc8a-817f786b043a", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93530" - } - ], - "active": true, - "name": { - "family": [ - "Doyle959" - ], - "given": [ - "Willis868" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "237 STATE ROAD" - ], - "city": "NORTH DARTMOUTH", - "state": "MA", - "postalCode": "2747", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f", - "resource": { - "resourceType": "Encounter", - "id": "659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a" - } - } - ], - "period": { - "start": "2010-05-14T19:52:39-07:00", - "end": "2010-05-14T20:07:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6bd6be0c-2cd0-493d-af74-32311a4a7563", - "resource": { - "resourceType": "Immunization", - "id": "6bd6be0c-2cd0-493d-af74-32311a4a7563", - "status": "completed", - "date": "2010-05-14T19:52:39-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:659abaf5-92bd-4ef2-bb74-fa2f1a3c4c0f" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:641b5155-f8b5-4845-b015-80d81aa480bc", - "resource": { - "resourceType": "Claim", - "id": "641b5155-f8b5-4845-b015-80d81aa480bc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de", - "resource": { - "resourceType": "Encounter", - "id": "ea9e3c13-3ad3-40df-9cdd-f3cd817944de", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2010-05-08T19:52:39-07:00", - "end": "2010-05-08T20:19:39-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1", - "resource": { - "resourceType": "Condition", - "id": "6f19df56-d88d-41d3-8d1a-a940a50e19e1", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" - }, - "dateRecorded": "2010-05-08", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-05-08T19:52:39-07:00", - "abatementDateTime": "2010-05-15T19:52:39-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f1b6cbda-e1fc-4547-88e4-0a8b9699a3bf", - "resource": { - "resourceType": "Procedure", - "id": "f1b6cbda-e1fc-4547-88e4-0a8b9699a3bf", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "reasonReference": { - "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" - }, - "performedPeriod": { - "start": "2010-05-08T19:52:39-07:00", - "end": "2010-05-08T20:04:39-07:00" - }, - "encounter": { - "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4353ffd8-d9e2-4538-ae2f-d026939a7431", - "resource": { - "resourceType": "MedicationOrder", - "id": "4353ffd8-d9e2-4538-ae2f-d026939a7431", - "dateWritten": "2010-05-08T19:52:39-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "prescriber": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - }, - "encounter": { - "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" - }, - "reasonReference": { - "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:ddf48aaa-8260-44b2-9519-883d0f3a18c7", - "resource": { - "resourceType": "Claim", - "id": "ddf48aaa-8260-44b2-9519-883d0f3a18c7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:4353ffd8-d9e2-4538-ae2f-d026939a7431" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d7cc9124-e4c3-4076-9a11-f72545d524ac", - "resource": { - "resourceType": "CarePlan", - "id": "d7cc9124-e4c3-4076-9a11-f72545d524ac", - "text": { - "status": "generated", - "div": "
    Respiratory therapy
    " - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "context": { - "reference": "urn:uuid:ea9e3c13-3ad3-40df-9cdd-f3cd817944de" - }, - "period": { - "start": "2010-05-08T19:52:39-07:00", - "end": "2010-11-05T19:52:39-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "addresses": [ - { - "reference": "urn:uuid:6f19df56-d88d-41d3-8d1a-a940a50e19e1" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:c179e666-caf6-41f6-9871-a1f148f3fb45", - "resource": { - "resourceType": "Claim", - "id": "c179e666-caf6-41f6-9871-a1f148f3fb45", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac", - "resource": { - "resourceType": "Encounter", - "id": "da87062c-3694-46e9-94a5-c09ca489b8ac", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2010-11-05T19:52:39-07:00", - "end": "2010-11-05T20:07:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8b1b3a3d-807f-463a-8d61-e282a732d0a8", - "resource": { - "resourceType": "Observation", - "id": "8b1b3a3d-807f-463a-8d61-e282a732d0a8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:43e738a5-a338-42d2-b130-76a484437e42", - "resource": { - "resourceType": "Observation", - "id": "43e738a5-a338-42d2-b130-76a484437e42", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:08fa42ee-365e-477e-b8ba-0c81ceb3154a", - "resource": { - "resourceType": "Observation", - "id": "08fa42ee-365e-477e-b8ba-0c81ceb3154a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2a38c5e-12c5-41d7-8109-a220f426492f", - "resource": { - "resourceType": "Observation", - "id": "a2a38c5e-12c5-41d7-8109-a220f426492f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f7fa66f-5509-4a78-b907-781587a39c8f", - "resource": { - "resourceType": "Observation", - "id": "9f7fa66f-5509-4a78-b907-781587a39c8f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:435f220c-64d2-4b34-aeae-80832cbbd085", - "resource": { - "resourceType": "Observation", - "id": "435f220c-64d2-4b34-aeae-80832cbbd085", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b249e0c-445c-4234-a15c-a59b08fc4f8f", - "resource": { - "resourceType": "Observation", - "id": "2b249e0c-445c-4234-a15c-a59b08fc4f8f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b89ebbea-7f9a-48f8-a9aa-167d53862888", - "resource": { - "resourceType": "Observation", - "id": "b89ebbea-7f9a-48f8-a9aa-167d53862888", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 80.020, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:52920ed8-3af5-4773-a3ac-6a7d4e7af012", - "resource": { - "resourceType": "Observation", - "id": "52920ed8-3af5-4773-a3ac-6a7d4e7af012", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 16.280, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d2cee4f-acb7-490b-a871-5e4d69c9dfb3", - "resource": { - "resourceType": "Observation", - "id": "3d2cee4f-acb7-490b-a871-5e4d69c9dfb3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 0.86000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53f78b14-7682-4a7b-9aae-8ec9cbcfd21c", - "resource": { - "resourceType": "Observation", - "id": "53f78b14-7682-4a7b-9aae-8ec9cbcfd21c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 9.9000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8998aa3-70fd-401c-bf72-d123b7953747", - "resource": { - "resourceType": "Observation", - "id": "e8998aa3-70fd-401c-bf72-d123b7953747", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 138.78, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8cc2fb6-7a8d-4229-b286-d7d8b2f8d98c", - "resource": { - "resourceType": "Observation", - "id": "b8cc2fb6-7a8d-4229-b286-d7d8b2f8d98c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 3.9800, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:02d268e7-fb6b-457a-a3ab-1bdf7389eb89", - "resource": { - "resourceType": "Observation", - "id": "02d268e7-fb6b-457a-a3ab-1bdf7389eb89", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 103.58, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e073447-f476-493f-9d37-179a11f58b31", - "resource": { - "resourceType": "Observation", - "id": "6e073447-f476-493f-9d37-179a11f58b31", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 25.740, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7fc8f159-8ffa-41a5-aa58-26520c7ecab1", - "resource": { - "resourceType": "Observation", - "id": "7fc8f159-8ffa-41a5-aa58-26520c7ecab1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 9.0808, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70121fba-da3f-4a1e-b612-f20a3cb6d042", - "resource": { - "resourceType": "Observation", - "id": "70121fba-da3f-4a1e-b612-f20a3cb6d042", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 4.9309, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:21e182da-dff6-441e-b0c2-f3a3b9b46f62", - "resource": { - "resourceType": "Observation", - "id": "21e182da-dff6-441e-b0c2-f3a3b9b46f62", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 16.191, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a84065c7-e904-4acc-ad51-9f39668f5695", - "resource": { - "resourceType": "Observation", - "id": "a84065c7-e904-4acc-ad51-9f39668f5695", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 45.164, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7276f2fb-37f4-4901-b075-53e748e67265", - "resource": { - "resourceType": "Observation", - "id": "7276f2fb-37f4-4901-b075-53e748e67265", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 80.755, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:001d3228-0bcf-43ba-af90-7d1c0fb22190", - "resource": { - "resourceType": "Observation", - "id": "001d3228-0bcf-43ba-af90-7d1c0fb22190", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 32.529, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4ad7fa9-a31c-421f-ac6c-032ecfa80f88", - "resource": { - "resourceType": "Observation", - "id": "d4ad7fa9-a31c-421f-ac6c-032ecfa80f88", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 33.172, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ff9d720-bd87-4a2b-b7e2-1182635b0c09", - "resource": { - "resourceType": "Observation", - "id": "1ff9d720-bd87-4a2b-b7e2-1182635b0c09", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 39.778, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29421a10-66db-4a10-b7fc-6e99bad82ef3", - "resource": { - "resourceType": "Observation", - "id": "29421a10-66db-4a10-b7fc-6e99bad82ef3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 200.68, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee462f37-2bc1-494a-9706-330cdb17267d", - "resource": { - "resourceType": "Observation", - "id": "ee462f37-2bc1-494a-9706-330cdb17267d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 360.68, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:411f3285-ed40-4b43-9288-565c8353f19b", - "resource": { - "resourceType": "Observation", - "id": "411f3285-ed40-4b43-9288-565c8353f19b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 11.053, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:97aedff2-37bb-4e9b-8ba2-441480c02f1a", - "resource": { - "resourceType": "Observation", - "id": "97aedff2-37bb-4e9b-8ba2-441480c02f1a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fe9119d8-fe40-4773-bf67-208dc746e5cc", - "resource": { - "resourceType": "Observation", - "id": "fe9119d8-fe40-4773-bf67-208dc746e5cc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "valueQuantity": { - "value": 6.3200, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:989350b4-5feb-4567-a916-41f6e36179dd", - "resource": { - "resourceType": "DiagnosticReport", - "id": "989350b4-5feb-4567-a916-41f6e36179dd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:6e073447-f476-493f-9d37-179a11f58b31", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:4522a9d2-fc57-49a3-9561-4e478e48f37c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4522a9d2-fc57-49a3-9561-4e478e48f37c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:da87062c-3694-46e9-94a5-c09ca489b8ac" - }, - "effectiveDateTime": "2010-11-05T19:52:39-07:00", - "issued": "2010-11-05T19:52:39.493-07:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:411f3285-ed40-4b43-9288-565c8353f19b", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:ca2d559e-05ed-43ea-8d51-3d281198d2be", - "resource": { - "resourceType": "Claim", - "id": "ca2d559e-05ed-43ea-8d51-3d281198d2be", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9", - "resource": { - "resourceType": "Encounter", - "id": "ab4d28a2-a2dd-48a0-b4ea-0184705382b9", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2011-11-11T18:52:39-08:00", - "end": "2011-11-11T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3dfedde6-6c6c-4ce8-91d4-a69c97cd29b1", - "resource": { - "resourceType": "Observation", - "id": "3dfedde6-6c6c-4ce8-91d4-a69c97cd29b1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:091b9bb5-514a-4b0a-b22e-ad8c8cfb3197", - "resource": { - "resourceType": "Observation", - "id": "091b9bb5-514a-4b0a-b22e-ad8c8cfb3197", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a315c78-e0fe-41d6-bd62-475f38830ad4", - "resource": { - "resourceType": "Observation", - "id": "3a315c78-e0fe-41d6-bd62-475f38830ad4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7ea95d9-0113-4f45-be07-cac688820b87", - "resource": { - "resourceType": "Observation", - "id": "a7ea95d9-0113-4f45-be07-cac688820b87", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71e8b565-18c8-46d4-ad26-32da02e3ddbb", - "resource": { - "resourceType": "Observation", - "id": "71e8b565-18c8-46d4-ad26-32da02e3ddbb", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 131, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7fd3111c-08ce-416e-be77-c5b2980272d0", - "resource": { - "resourceType": "Observation", - "id": "7fd3111c-08ce-416e-be77-c5b2980272d0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 67, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c19d44e0-1f51-4536-bd8b-8b048b59725f", - "resource": { - "resourceType": "Observation", - "id": "c19d44e0-1f51-4536-bd8b-8b048b59725f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7f3898da-13ec-487a-ab99-dbe319b2a766", - "resource": { - "resourceType": "Observation", - "id": "7f3898da-13ec-487a-ab99-dbe319b2a766", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 95.5, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:658cf76a-f850-4861-ad91-7ce546b57af9", - "resource": { - "resourceType": "Observation", - "id": "658cf76a-f850-4861-ad91-7ce546b57af9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 12.450, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:615f8813-b216-43d8-8d41-a818bdd2273a", - "resource": { - "resourceType": "Observation", - "id": "615f8813-b216-43d8-8d41-a818bdd2273a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.80000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:575089a7-49ca-4597-9320-2ac878aa9f9a", - "resource": { - "resourceType": "Observation", - "id": "575089a7-49ca-4597-9320-2ac878aa9f9a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.5400, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8e567f2-47ab-4159-ba36-e7b3e07834f4", - "resource": { - "resourceType": "Observation", - "id": "d8e567f2-47ab-4159-ba36-e7b3e07834f4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 141.62, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:222e6976-7948-4231-af06-de764fd8c6b8", - "resource": { - "resourceType": "Observation", - "id": "222e6976-7948-4231-af06-de764fd8c6b8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.3100, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d3fca14-4f95-43dd-a886-c13b5f046096", - "resource": { - "resourceType": "Observation", - "id": "6d3fca14-4f95-43dd-a886-c13b5f046096", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 102.16, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c3ac129a-5767-42f2-8605-01516a1cb7e0", - "resource": { - "resourceType": "Observation", - "id": "c3ac129a-5767-42f2-8605-01516a1cb7e0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 21.610, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dcc3146f-9369-47f4-ba13-6f92bbbffa98", - "resource": { - "resourceType": "Observation", - "id": "dcc3146f-9369-47f4-ba13-6f92bbbffa98", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9aaadfd6-2c24-4442-bf5d-20452e200f7a", - "resource": { - "resourceType": "Observation", - "id": "9aaadfd6-2c24-4442-bf5d-20452e200f7a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "valueQuantity": { - "value": 6, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c6227e1-508f-481c-b374-40c62af3f271", - "resource": { - "resourceType": "Immunization", - "id": "3c6227e1-508f-481c-b374-40c62af3f271", - "status": "completed", - "date": "2011-11-11T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:6a0919b3-c980-40ae-9ba5-96e4ad156c4d", - "resource": { - "resourceType": "DiagnosticReport", - "id": "6a0919b3-c980-40ae-9ba5-96e4ad156c4d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:ab4d28a2-a2dd-48a0-b4ea-0184705382b9" - }, - "effectiveDateTime": "2011-11-11T18:52:39-08:00", - "issued": "2011-11-11T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:c3ac129a-5767-42f2-8605-01516a1cb7e0", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b2dcf71d-3448-45f1-9445-e6bae561611b", - "resource": { - "resourceType": "Claim", - "id": "b2dcf71d-3448-45f1-9445-e6bae561611b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486", - "resource": { - "resourceType": "Encounter", - "id": "670309a3-be34-417a-8d78-13b8d1ca2486", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2012-11-16T18:52:39-08:00", - "end": "2012-11-16T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4f2afec2-c2b6-47d5-a05d-aaca9787eef6", - "resource": { - "resourceType": "Observation", - "id": "4f2afec2-c2b6-47d5-a05d-aaca9787eef6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ba4ee99-1274-4aea-a766-e82c74267faf", - "resource": { - "resourceType": "Observation", - "id": "2ba4ee99-1274-4aea-a766-e82c74267faf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:226b2507-471b-41da-9021-0210f0de940e", - "resource": { - "resourceType": "Observation", - "id": "226b2507-471b-41da-9021-0210f0de940e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6933e334-8d1c-4121-87d9-63e263a8b641", - "resource": { - "resourceType": "Observation", - "id": "6933e334-8d1c-4121-87d9-63e263a8b641", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70afdfbf-f71f-425e-bc08-ce46996c8557", - "resource": { - "resourceType": "Observation", - "id": "70afdfbf-f71f-425e-bc08-ce46996c8557", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 101, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bcd9105a-7d96-4af8-8795-645a0e6a8493", - "resource": { - "resourceType": "Observation", - "id": "bcd9105a-7d96-4af8-8795-645a0e6a8493", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 85, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:787dbac8-ad30-45ab-a1af-e96f19c435ae", - "resource": { - "resourceType": "Observation", - "id": "787dbac8-ad30-45ab-a1af-e96f19c435ae", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df7f409c-d231-422d-b039-73dd61efbe75", - "resource": { - "resourceType": "Observation", - "id": "df7f409c-d231-422d-b039-73dd61efbe75", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 75.810, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3d2924a-e4da-4b42-9d77-baba07de69a6", - "resource": { - "resourceType": "Observation", - "id": "f3d2924a-e4da-4b42-9d77-baba07de69a6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 8.8300, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a9dcf68-fdd2-41d4-a3da-cec3111093e4", - "resource": { - "resourceType": "Observation", - "id": "3a9dcf68-fdd2-41d4-a3da-cec3111093e4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.79000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b37615d-d752-42d8-9ecd-5592ecddfe21", - "resource": { - "resourceType": "Observation", - "id": "8b37615d-d752-42d8-9ecd-5592ecddfe21", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.7900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9727c9a7-bcf0-496d-9cde-11a9b0f6221d", - "resource": { - "resourceType": "Observation", - "id": "9727c9a7-bcf0-496d-9cde-11a9b0f6221d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 139.39, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2083699-11a0-483a-ba2f-5a6c958af14b", - "resource": { - "resourceType": "Observation", - "id": "b2083699-11a0-483a-ba2f-5a6c958af14b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.9200, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:404a813a-e432-4250-a06c-e7718ddcc4be", - "resource": { - "resourceType": "Observation", - "id": "404a813a-e432-4250-a06c-e7718ddcc4be", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 105, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3916a2b-1805-4264-84c1-11c1662bb295", - "resource": { - "resourceType": "Observation", - "id": "b3916a2b-1805-4264-84c1-11c1662bb295", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 27.990, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ce62e32-1557-49f7-a397-5f2e78379659", - "resource": { - "resourceType": "Observation", - "id": "6ce62e32-1557-49f7-a397-5f2e78379659", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 164.71, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82929ea8-e275-4c33-b62e-483ecd08d995", - "resource": { - "resourceType": "Observation", - "id": "82929ea8-e275-4c33-b62e-483ecd08d995", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 102.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8abd3258-d1ad-40a8-99ab-fd8deb1e9844", - "resource": { - "resourceType": "Observation", - "id": "8abd3258-d1ad-40a8-99ab-fd8deb1e9844", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 75.650, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3da90c1a-0a99-4b7c-9710-8a38c8d97e45", - "resource": { - "resourceType": "Observation", - "id": "3da90c1a-0a99-4b7c-9710-8a38c8d97e45", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 68.510, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:082e903a-1466-4702-b881-9e0465ed88ea", - "resource": { - "resourceType": "Observation", - "id": "082e903a-1466-4702-b881-9e0465ed88ea", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f270ddac-c2b0-45bd-82c9-6c6ea4462fb6", - "resource": { - "resourceType": "Observation", - "id": "f270ddac-c2b0-45bd-82c9-6c6ea4462fb6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "valueQuantity": { - "value": 6.2800, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d128e8a-5a6d-47ff-8a83-8fa7f98c9cb1", - "resource": { - "resourceType": "Immunization", - "id": "9d128e8a-5a6d-47ff-8a83-8fa7f98c9cb1", - "status": "completed", - "date": "2012-11-16T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:65d5da8c-bb67-470b-b070-65f85e921dfc", - "resource": { - "resourceType": "DiagnosticReport", - "id": "65d5da8c-bb67-470b-b070-65f85e921dfc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:b3916a2b-1805-4264-84c1-11c1662bb295", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:d1b936cf-567e-4591-93ad-6fb29c2478fa", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d1b936cf-567e-4591-93ad-6fb29c2478fa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:670309a3-be34-417a-8d78-13b8d1ca2486" - }, - "effectiveDateTime": "2012-11-16T18:52:39-08:00", - "issued": "2012-11-16T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:3da90c1a-0a99-4b7c-9710-8a38c8d97e45", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:ff296cf4-dc5a-4dee-b556-055cb4d59325", - "resource": { - "resourceType": "Claim", - "id": "ff296cf4-dc5a-4dee-b556-055cb4d59325", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788", - "resource": { - "resourceType": "Encounter", - "id": "011ec61e-e99b-4ffe-a5c3-b6f28020f788", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2013-11-22T18:52:39-08:00", - "end": "2013-11-22T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:af0c6c9d-ed83-4a58-9258-bfb6223e715b", - "resource": { - "resourceType": "Observation", - "id": "af0c6c9d-ed83-4a58-9258-bfb6223e715b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8c1cc80d-217d-413e-a182-cbcf73ced89b", - "resource": { - "resourceType": "Observation", - "id": "8c1cc80d-217d-413e-a182-cbcf73ced89b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:672c6bd6-fbfa-476d-b955-7763b0ca5b51", - "resource": { - "resourceType": "Observation", - "id": "672c6bd6-fbfa-476d-b955-7763b0ca5b51", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:63a2d450-7607-422b-81d8-e01dc856db60", - "resource": { - "resourceType": "Observation", - "id": "63a2d450-7607-422b-81d8-e01dc856db60", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a999933d-7a0b-4129-b92c-b47715840c10", - "resource": { - "resourceType": "Observation", - "id": "a999933d-7a0b-4129-b92c-b47715840c10", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7f8b06c1-cf34-4473-9879-30edfb70e009", - "resource": { - "resourceType": "Observation", - "id": "7f8b06c1-cf34-4473-9879-30edfb70e009", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 68, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c3c2dc96-a3af-46a5-ab19-88027cf20680", - "resource": { - "resourceType": "Observation", - "id": "c3c2dc96-a3af-46a5-ab19-88027cf20680", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9a6113db-31c6-43be-b20b-a81b92ff4891", - "resource": { - "resourceType": "Observation", - "id": "9a6113db-31c6-43be-b20b-a81b92ff4891", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 71.610, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfd84a6d-9c18-4040-8179-16cc385cac6e", - "resource": { - "resourceType": "Observation", - "id": "cfd84a6d-9c18-4040-8179-16cc385cac6e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 10.160, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e835fffe-4304-4967-9aea-506bfb3b83d8", - "resource": { - "resourceType": "Observation", - "id": "e835fffe-4304-4967-9aea-506bfb3b83d8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 1.0200, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5632b342-89c2-40dd-a00a-9bd866d2f503", - "resource": { - "resourceType": "Observation", - "id": "5632b342-89c2-40dd-a00a-9bd866d2f503", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.3000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0bc0e869-58d3-4ad4-abdf-0a96cec68053", - "resource": { - "resourceType": "Observation", - "id": "0bc0e869-58d3-4ad4-abdf-0a96cec68053", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 141.72, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:54e932c8-ea30-4135-9b5c-1b4dfdfaa131", - "resource": { - "resourceType": "Observation", - "id": "54e932c8-ea30-4135-9b5c-1b4dfdfaa131", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.1600, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfb92202-11de-4434-92b3-103728dfae5e", - "resource": { - "resourceType": "Observation", - "id": "cfb92202-11de-4434-92b3-103728dfae5e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 101.73, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c5de72f9-db83-4f60-adfb-a4d1df56bd86", - "resource": { - "resourceType": "Observation", - "id": "c5de72f9-db83-4f60-adfb-a4d1df56bd86", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 22.640, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8dba8b3-11f2-45ec-a5d9-9ef30d22dc18", - "resource": { - "resourceType": "Observation", - "id": "b8dba8b3-11f2-45ec-a5d9-9ef30d22dc18", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:07b11457-1de7-41fb-a472-47de4dd42564", - "resource": { - "resourceType": "Observation", - "id": "07b11457-1de7-41fb-a472-47de4dd42564", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "valueQuantity": { - "value": 6.1600, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40283994-a00b-41a4-b910-9e08dc302acf", - "resource": { - "resourceType": "Immunization", - "id": "40283994-a00b-41a4-b910-9e08dc302acf", - "status": "completed", - "date": "2013-11-22T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:595336ba-d00d-4061-a4e4-7f288a552add", - "resource": { - "resourceType": "DiagnosticReport", - "id": "595336ba-d00d-4061-a4e4-7f288a552add", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:011ec61e-e99b-4ffe-a5c3-b6f28020f788" - }, - "effectiveDateTime": "2013-11-22T18:52:39-08:00", - "issued": "2013-11-22T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:c5de72f9-db83-4f60-adfb-a4d1df56bd86", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b2714247-19be-41c2-b878-91bd9bf24e52", - "resource": { - "resourceType": "Claim", - "id": "b2714247-19be-41c2-b878-91bd9bf24e52", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313", - "resource": { - "resourceType": "Encounter", - "id": "950f6970-4fe6-4205-bba3-7be621435313", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2014-03-17T19:52:39-07:00", - "end": "2014-03-17T20:07:39-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fbc36a68-14be-4eba-9c54-09c0968f9dbc", - "resource": { - "resourceType": "Condition", - "id": "fbc36a68-14be-4eba-9c54-09c0968f9dbc", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313" - }, - "dateRecorded": "2014-03-17", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2014-03-17T19:52:39-07:00", - "abatementDateTime": "2014-03-28T19:52:39-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:4d70c627-0881-4c1d-b1d3-a0f1e49d1566", - "resource": { - "resourceType": "Observation", - "id": "4d70c627-0881-4c1d-b1d3-a0f1e49d1566", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:950f6970-4fe6-4205-bba3-7be621435313" - }, - "effectiveDateTime": "2014-03-17T19:52:39-07:00", - "issued": "2014-03-17T19:52:39.493-07:00", - "valueQuantity": { - "value": 37.283, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:58d48e5e-86e6-4598-99f7-4bb6fbda952e", - "resource": { - "resourceType": "Claim", - "id": "58d48e5e-86e6-4598-99f7-4bb6fbda952e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189", - "resource": { - "resourceType": "Encounter", - "id": "1823d974-5838-40b6-acc2-a1674c659189", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2014-11-28T18:52:39-08:00", - "end": "2014-11-28T19:22:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1f3be8cd-4779-477d-b3e5-8301376b3c2d", - "resource": { - "resourceType": "Observation", - "id": "1f3be8cd-4779-477d-b3e5-8301376b3c2d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47467834-5b7d-45bf-bdb5-7f9fed78242a", - "resource": { - "resourceType": "Observation", - "id": "47467834-5b7d-45bf-bdb5-7f9fed78242a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:976cd9e8-bfd6-4222-aa8c-3756da9ababe", - "resource": { - "resourceType": "Observation", - "id": "976cd9e8-bfd6-4222-aa8c-3756da9ababe", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:42911871-bd0b-484a-a603-fb5da39d120c", - "resource": { - "resourceType": "Observation", - "id": "42911871-bd0b-484a-a603-fb5da39d120c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f8f32e7-006b-487a-aa6f-539cfa9f5203", - "resource": { - "resourceType": "Observation", - "id": "5f8f32e7-006b-487a-aa6f-539cfa9f5203", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6cf975b-7aa0-4f85-bf7f-b502abdeadbe", - "resource": { - "resourceType": "Observation", - "id": "b6cf975b-7aa0-4f85-bf7f-b502abdeadbe", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53507506-838b-433c-8916-7e0f4e6279ee", - "resource": { - "resourceType": "Observation", - "id": "53507506-838b-433c-8916-7e0f4e6279ee", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2ff4d3e-015f-4668-a207-48aaf1916d47", - "resource": { - "resourceType": "Observation", - "id": "b2ff4d3e-015f-4668-a207-48aaf1916d47", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 88.860, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4459a77-32fd-468f-8658-623c067748c4", - "resource": { - "resourceType": "Observation", - "id": "d4459a77-32fd-468f-8658-623c067748c4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 10.870, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:349b712c-d8ad-4ac8-8b2c-937af8c37274", - "resource": { - "resourceType": "Observation", - "id": "349b712c-d8ad-4ac8-8b2c-937af8c37274", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.97000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ed99f79-1036-4c8b-adc9-620519d809ba", - "resource": { - "resourceType": "Observation", - "id": "6ed99f79-1036-4c8b-adc9-620519d809ba", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.4400, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47355ffc-38d7-4027-af0a-885bad94b53b", - "resource": { - "resourceType": "Observation", - "id": "47355ffc-38d7-4027-af0a-885bad94b53b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 137.21, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ae3afdcb-7a3e-4d12-bb74-7b59bbe6dfd3", - "resource": { - "resourceType": "Observation", - "id": "ae3afdcb-7a3e-4d12-bb74-7b59bbe6dfd3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.6100, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50c7e3ef-8731-4795-a3df-37b151388a09", - "resource": { - "resourceType": "Observation", - "id": "50c7e3ef-8731-4795-a3df-37b151388a09", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 108.45, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", - "resource": { - "resourceType": "Observation", - "id": "0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 22.850, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82c756b4-052a-4f7f-989e-8b8b508d3ccf", - "resource": { - "resourceType": "Observation", - "id": "82c756b4-052a-4f7f-989e-8b8b508d3ccf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3aa43a9b-1fca-4e34-9a14-c48892499715", - "resource": { - "resourceType": "Observation", - "id": "3aa43a9b-1fca-4e34-9a14-c48892499715", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "valueQuantity": { - "value": 6.0600, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a38b30bc-b795-4f4f-adf6-02700db3b80c", - "resource": { - "resourceType": "Procedure", - "id": "a38b30bc-b795-4f4f-adf6-02700db3b80c", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2014-11-28T18:52:39-08:00", - "end": "2014-11-28T19:07:39-08:00" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:53ac6448-eb28-458e-b036-087ae17f31bf", - "resource": { - "resourceType": "Immunization", - "id": "53ac6448-eb28-458e-b036-087ae17f31bf", - "status": "completed", - "date": "2014-11-28T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c8a80d5d-f63d-4419-9104-ee22c6d867b9", - "resource": { - "resourceType": "Immunization", - "id": "c8a80d5d-f63d-4419-9104-ee22c6d867b9", - "status": "completed", - "date": "2014-11-28T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:14ece09e-a0e7-4dd1-8738-3e513ef89bf4", - "resource": { - "resourceType": "DiagnosticReport", - "id": "14ece09e-a0e7-4dd1-8738-3e513ef89bf4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:1823d974-5838-40b6-acc2-a1674c659189" - }, - "effectiveDateTime": "2014-11-28T18:52:39-08:00", - "issued": "2014-11-28T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:0427cf6b-94e1-49f8-ad6e-930d3e50bfc5", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0a38c1c9-20ab-458c-82c1-f8e77b071720", - "resource": { - "resourceType": "Claim", - "id": "0a38c1c9-20ab-458c-82c1-f8e77b071720", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 430.76, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c", - "resource": { - "resourceType": "Encounter", - "id": "d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2015-01-18T18:52:39-08:00", - "end": "2015-01-18T19:37:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b1e119c2-1643-4099-92ce-9e95d8942b2b", - "resource": { - "resourceType": "Procedure", - "id": "b1e119c2-1643-4099-92ce-9e95d8942b2b", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "performedPeriod": { - "start": "2015-01-18T18:52:39-08:00", - "end": "2015-01-18T19:22:39-08:00" - }, - "encounter": { - "reference": "urn:uuid:d2c9a2ed-a26a-4b7f-8e57-f84b0d7bd23c" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:db0f1238-1920-44a6-98ec-5a83648c1ed8", - "resource": { - "resourceType": "Claim", - "id": "db0f1238-1920-44a6-98ec-5a83648c1ed8", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - }, - "net": { - "value": 21085.78, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847", - "resource": { - "resourceType": "Encounter", - "id": "01bfb335-6313-4898-8ab8-0e461e5bb847", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2015-12-04T18:52:39-08:00", - "end": "2015-12-04T19:22:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:99323959-640c-4e5c-96fe-c2e6bd099513", - "resource": { - "resourceType": "Observation", - "id": "99323959-640c-4e5c-96fe-c2e6bd099513", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c2e9798f-b55f-4e1f-adba-1bec89e3f699", - "resource": { - "resourceType": "Observation", - "id": "c2e9798f-b55f-4e1f-adba-1bec89e3f699", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f471de46-c10a-47ee-942e-57765eea248a", - "resource": { - "resourceType": "Observation", - "id": "f471de46-c10a-47ee-942e-57765eea248a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca903c50-c4ab-47f8-8244-088405cf5977", - "resource": { - "resourceType": "Observation", - "id": "ca903c50-c4ab-47f8-8244-088405cf5977", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b32a170e-8ad0-4101-99c0-dd13459da450", - "resource": { - "resourceType": "Observation", - "id": "b32a170e-8ad0-4101-99c0-dd13459da450", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e13dda94-5e79-4666-9719-8d8bcc23a158", - "resource": { - "resourceType": "Observation", - "id": "e13dda94-5e79-4666-9719-8d8bcc23a158", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e80a2fd3-e367-4211-8132-3b8ca1d8cd22", - "resource": { - "resourceType": "Observation", - "id": "e80a2fd3-e367-4211-8132-3b8ca1d8cd22", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ad4d891-dc2d-42d2-bfc9-2bca5eb3a05e", - "resource": { - "resourceType": "Observation", - "id": "5ad4d891-dc2d-42d2-bfc9-2bca5eb3a05e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 70.200, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c26e5b59-394d-4662-a62e-77ef3f9d96ab", - "resource": { - "resourceType": "Observation", - "id": "c26e5b59-394d-4662-a62e-77ef3f9d96ab", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 18.720, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddca3175-5a43-49b9-bfde-c4425efa81dc", - "resource": { - "resourceType": "Observation", - "id": "ddca3175-5a43-49b9-bfde-c4425efa81dc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.76000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4165197d-501f-4753-9c91-5a46c07fb0fa", - "resource": { - "resourceType": "Observation", - "id": "4165197d-501f-4753-9c91-5a46c07fb0fa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 8.5700, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2be499cc-d580-4d23-b32c-87159a09f7a0", - "resource": { - "resourceType": "Observation", - "id": "2be499cc-d580-4d23-b32c-87159a09f7a0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 141.15, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67ea8ba3-0169-4499-8c32-1e2ef07e3565", - "resource": { - "resourceType": "Observation", - "id": "67ea8ba3-0169-4499-8c32-1e2ef07e3565", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.0100, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47e1ed26-65c3-47b9-b907-232b2c667189", - "resource": { - "resourceType": "Observation", - "id": "47e1ed26-65c3-47b9-b907-232b2c667189", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 104.60, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:802da8b4-db40-4b00-9c4b-35d3a0056cfe", - "resource": { - "resourceType": "Observation", - "id": "802da8b4-db40-4b00-9c4b-35d3a0056cfe", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 23.390, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e78b7d8-3b20-4b02-9946-f7187d3d0090", - "resource": { - "resourceType": "Observation", - "id": "7e78b7d8-3b20-4b02-9946-f7187d3d0090", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 195.21, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:651d9a01-8dfa-4c78-a49d-b6b4c4a7a85c", - "resource": { - "resourceType": "Observation", - "id": "651d9a01-8dfa-4c78-a49d-b6b4c4a7a85c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 111.03, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbe2464b-b19e-4c6d-8425-a5f6a20399c1", - "resource": { - "resourceType": "Observation", - "id": "bbe2464b-b19e-4c6d-8425-a5f6a20399c1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 106.03, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9681baf6-4b51-4144-965c-9c96314821ae", - "resource": { - "resourceType": "Observation", - "id": "9681baf6-4b51-4144-965c-9c96314821ae", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 66.970, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:356edd60-4206-4e34-b6e2-1ce68bd8c693", - "resource": { - "resourceType": "Observation", - "id": "356edd60-4206-4e34-b6e2-1ce68bd8c693", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 3.8256, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa94d2bf-a78c-4718-b8cb-163dfe7ce830", - "resource": { - "resourceType": "Observation", - "id": "fa94d2bf-a78c-4718-b8cb-163dfe7ce830", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.1438, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb83beb7-9084-49e4-8cc0-6c83a8f7ef27", - "resource": { - "resourceType": "Observation", - "id": "cb83beb7-9084-49e4-8cc0-6c83a8f7ef27", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 17.086, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7234c947-45c7-4ba3-adae-6a5a0dda91f2", - "resource": { - "resourceType": "Observation", - "id": "7234c947-45c7-4ba3-adae-6a5a0dda91f2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 40.758, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d14adbcb-bd94-4985-9878-76b461cd5516", - "resource": { - "resourceType": "Observation", - "id": "d14adbcb-bd94-4985-9878-76b461cd5516", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 84.708, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f0205de1-9351-42fe-901e-c7ab23fb24a1", - "resource": { - "resourceType": "Observation", - "id": "f0205de1-9351-42fe-901e-c7ab23fb24a1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 27.499, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5b801394-0d88-48d6-9864-696d3fb6a22a", - "resource": { - "resourceType": "Observation", - "id": "5b801394-0d88-48d6-9864-696d3fb6a22a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 34.677, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8fca8466-132c-4377-ab19-03557741684b", - "resource": { - "resourceType": "Observation", - "id": "8fca8466-132c-4377-ab19-03557741684b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 40.780, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:796a0b69-d993-4936-b606-62b4ff53d7f7", - "resource": { - "resourceType": "Observation", - "id": "796a0b69-d993-4936-b606-62b4ff53d7f7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 266.53, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8ea379cd-681a-43c1-a72e-46599a656369", - "resource": { - "resourceType": "Observation", - "id": "8ea379cd-681a-43c1-a72e-46599a656369", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 253.01, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8f5bb42a-e434-40c5-bfb8-3407a226cc0a", - "resource": { - "resourceType": "Observation", - "id": "8f5bb42a-e434-40c5-bfb8-3407a226cc0a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 10.838, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aea84217-58b6-4cb5-b1b3-ead39313a8fc", - "resource": { - "resourceType": "Observation", - "id": "aea84217-58b6-4cb5-b1b3-ead39313a8fc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ef2f503-3ef7-488c-8a57-cd4f82a9f85b", - "resource": { - "resourceType": "Observation", - "id": "9ef2f503-3ef7-488c-8a57-cd4f82a9f85b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "valueQuantity": { - "value": 5.9300, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a8518237-a0ef-4ebf-9735-cc01f0ddc276", - "resource": { - "resourceType": "Procedure", - "id": "a8518237-a0ef-4ebf-9735-cc01f0ddc276", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2015-12-04T18:52:39-08:00", - "end": "2015-12-04T19:07:39-08:00" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f74727ba-81ed-4ad4-b6a5-764ed865a6fc", - "resource": { - "resourceType": "Immunization", - "id": "f74727ba-81ed-4ad4-b6a5-764ed865a6fc", - "status": "completed", - "date": "2015-12-04T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:513034fd-ba93-400c-9ecc-698b450c3ff1", - "resource": { - "resourceType": "DiagnosticReport", - "id": "513034fd-ba93-400c-9ecc-698b450c3ff1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:802da8b4-db40-4b00-9c4b-35d3a0056cfe", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:854138ea-366e-40a6-9a4d-6b6bde7425f1", - "resource": { - "resourceType": "DiagnosticReport", - "id": "854138ea-366e-40a6-9a4d-6b6bde7425f1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:9681baf6-4b51-4144-965c-9c96314821ae", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:30438cdb-e42c-4b8b-a101-2a85ee93f30c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "30438cdb-e42c-4b8b-a101-2a85ee93f30c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:01bfb335-6313-4898-8ab8-0e461e5bb847" - }, - "effectiveDateTime": "2015-12-04T18:52:39-08:00", - "issued": "2015-12-04T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:8f5bb42a-e434-40c5-bfb8-3407a226cc0a", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:982a470b-06d6-472d-8642-16f022877d46", - "resource": { - "resourceType": "Claim", - "id": "982a470b-06d6-472d-8642-16f022877d46", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 502.94, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a", - "resource": { - "resourceType": "Encounter", - "id": "37444168-0363-4600-ab6b-b4e0088e768a", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2016-12-09T18:52:39-08:00", - "end": "2016-12-09T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6e39bfdc-b174-47be-bbf5-5df749a688d5", - "resource": { - "resourceType": "Observation", - "id": "6e39bfdc-b174-47be-bbf5-5df749a688d5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0300e321-3e6b-42a4-b9f7-2d88e7e1f064", - "resource": { - "resourceType": "Observation", - "id": "0300e321-3e6b-42a4-b9f7-2d88e7e1f064", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66773dbc-f8f7-4aa2-87de-0b254a619292", - "resource": { - "resourceType": "Observation", - "id": "66773dbc-f8f7-4aa2-87de-0b254a619292", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:85d4484b-1080-4a9c-b719-ae0980a75b47", - "resource": { - "resourceType": "Observation", - "id": "85d4484b-1080-4a9c-b719-ae0980a75b47", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b84ef9ef-d30b-4e2e-b85a-14668e26b597", - "resource": { - "resourceType": "Observation", - "id": "b84ef9ef-d30b-4e2e-b85a-14668e26b597", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 87, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ea15b6f4-f751-4627-b18f-f910e36e8426", - "resource": { - "resourceType": "Observation", - "id": "ea15b6f4-f751-4627-b18f-f910e36e8426", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 92, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:141de97b-1235-47b4-99fc-15ddb63e75e6", - "resource": { - "resourceType": "Observation", - "id": "141de97b-1235-47b4-99fc-15ddb63e75e6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:705eefa9-baf2-4245-8292-36c5c4e6349b", - "resource": { - "resourceType": "Observation", - "id": "705eefa9-baf2-4245-8292-36c5c4e6349b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 81.430, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9558c234-0562-48f0-92e4-c596c2b6a90e", - "resource": { - "resourceType": "Observation", - "id": "9558c234-0562-48f0-92e4-c596c2b6a90e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 14.080, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35cb1a99-71a0-47be-8e23-1bd70a79dff7", - "resource": { - "resourceType": "Observation", - "id": "35cb1a99-71a0-47be-8e23-1bd70a79dff7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 1.0700, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2647f5fd-e871-4d3b-bb29-3b7e41701208", - "resource": { - "resourceType": "Observation", - "id": "2647f5fd-e871-4d3b-bb29-3b7e41701208", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 8.6900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:585c9a48-d551-45a3-92b5-7009c801d000", - "resource": { - "resourceType": "Observation", - "id": "585c9a48-d551-45a3-92b5-7009c801d000", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 141.85, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a6e89c79-e52e-473f-bd3e-5f0e30571590", - "resource": { - "resourceType": "Observation", - "id": "a6e89c79-e52e-473f-bd3e-5f0e30571590", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.2300, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:348e3f9e-5841-429e-9c62-0b1cec36e6d5", - "resource": { - "resourceType": "Observation", - "id": "348e3f9e-5841-429e-9c62-0b1cec36e6d5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 106.70, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", - "resource": { - "resourceType": "Observation", - "id": "1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 24.660, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd46f602-9656-454d-8423-70b63e1f5b97", - "resource": { - "resourceType": "Observation", - "id": "dd46f602-9656-454d-8423-70b63e1f5b97", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:efd5875c-ece5-4724-89fa-434121fe45cb", - "resource": { - "resourceType": "Observation", - "id": "efd5875c-ece5-4724-89fa-434121fe45cb", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "valueQuantity": { - "value": 5.8500, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf27f8d0-6820-41d5-8878-01471bdb4b43", - "resource": { - "resourceType": "Immunization", - "id": "cf27f8d0-6820-41d5-8878-01471bdb4b43", - "status": "completed", - "date": "2016-12-09T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:72440146-8e9b-4b58-ae8f-9533f3265e2a", - "resource": { - "resourceType": "DiagnosticReport", - "id": "72440146-8e9b-4b58-ae8f-9533f3265e2a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:37444168-0363-4600-ab6b-b4e0088e768a" - }, - "effectiveDateTime": "2016-12-09T18:52:39-08:00", - "issued": "2016-12-09T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:1b8f94d8-dd47-43b6-a8f9-7ac0d5823ab5", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:307ff1d2-2101-4760-beb8-385b9a1a5acb", - "resource": { - "resourceType": "Claim", - "id": "307ff1d2-2101-4760-beb8-385b9a1a5acb", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356", - "resource": { - "resourceType": "Encounter", - "id": "19d53601-0ca8-4316-9435-40265d2f9356", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2017-12-15T18:52:39-08:00", - "end": "2017-12-15T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b40a71dd-1cb6-4c88-a626-176437e4f810", - "resource": { - "resourceType": "Observation", - "id": "b40a71dd-1cb6-4c88-a626-176437e4f810", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:732bb789-7c07-4bdf-805d-f6cd44a0a62a", - "resource": { - "resourceType": "Observation", - "id": "732bb789-7c07-4bdf-805d-f6cd44a0a62a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ce32e9ed-59c5-4799-9e64-e37d0d4fc188", - "resource": { - "resourceType": "Observation", - "id": "ce32e9ed-59c5-4799-9e64-e37d0d4fc188", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:10e191a6-f2fe-4c8b-9c3e-ce145ba0c8b9", - "resource": { - "resourceType": "Observation", - "id": "10e191a6-f2fe-4c8b-9c3e-ce145ba0c8b9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ea326672-4519-4534-8645-0272fcd3f1df", - "resource": { - "resourceType": "Observation", - "id": "ea326672-4519-4534-8645-0272fcd3f1df", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 118, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aea79c24-32ce-4a4b-a18a-4a94f3187a87", - "resource": { - "resourceType": "Observation", - "id": "aea79c24-32ce-4a4b-a18a-4a94f3187a87", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:13b57867-9f2b-472b-9e27-8a7e300c14c4", - "resource": { - "resourceType": "Observation", - "id": "13b57867-9f2b-472b-9e27-8a7e300c14c4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5dc175ad-a4ce-412c-ad98-dca0a3aecda1", - "resource": { - "resourceType": "Observation", - "id": "5dc175ad-a4ce-412c-ad98-dca0a3aecda1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 75.900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99e923b3-ffe9-4dd6-9871-22eabbf599d1", - "resource": { - "resourceType": "Observation", - "id": "99e923b3-ffe9-4dd6-9871-22eabbf599d1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 17.090, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fbabe4df-6b2e-414d-8b79-97e1dc491f5f", - "resource": { - "resourceType": "Observation", - "id": "fbabe4df-6b2e-414d-8b79-97e1dc491f5f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.74000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:86aa7ee4-407f-43d0-9fdf-571858dd7ebc", - "resource": { - "resourceType": "Observation", - "id": "86aa7ee4-407f-43d0-9fdf-571858dd7ebc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.5300, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:be50d438-9336-44ec-9528-2a437941c1c7", - "resource": { - "resourceType": "Observation", - "id": "be50d438-9336-44ec-9528-2a437941c1c7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 139.55, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9be8fdb-d8ed-4a4c-a9c3-9ae1c4cb1317", - "resource": { - "resourceType": "Observation", - "id": "a9be8fdb-d8ed-4a4c-a9c3-9ae1c4cb1317", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.3300, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7ce122f8-9814-41ba-b23f-241f47fa0ca7", - "resource": { - "resourceType": "Observation", - "id": "7ce122f8-9814-41ba-b23f-241f47fa0ca7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 108.81, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32619173-042b-4fae-8c5e-8577e1583f19", - "resource": { - "resourceType": "Observation", - "id": "32619173-042b-4fae-8c5e-8577e1583f19", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 21.960, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ada54f9-ba2a-49ab-9fff-1c4c6dfe43d4", - "resource": { - "resourceType": "Observation", - "id": "5ada54f9-ba2a-49ab-9fff-1c4c6dfe43d4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d625ed0e-6efd-430d-a858-b604a027f7a8", - "resource": { - "resourceType": "Observation", - "id": "d625ed0e-6efd-430d-a858-b604a027f7a8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "valueQuantity": { - "value": 6.3300, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1275a54a-8321-44fc-84c7-8eeb39188b31", - "resource": { - "resourceType": "Immunization", - "id": "1275a54a-8321-44fc-84c7-8eeb39188b31", - "status": "completed", - "date": "2017-12-15T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7a20af2f-3f74-4af0-8a81-df0e2dfddd17", - "resource": { - "resourceType": "DiagnosticReport", - "id": "7a20af2f-3f74-4af0-8a81-df0e2dfddd17", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:19d53601-0ca8-4316-9435-40265d2f9356" - }, - "effectiveDateTime": "2017-12-15T18:52:39-08:00", - "issued": "2017-12-15T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:32619173-042b-4fae-8c5e-8577e1583f19", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b9ab9708-2f0e-4ba1-a64c-324bddcdfc72", - "resource": { - "resourceType": "Claim", - "id": "b9ab9708-2f0e-4ba1-a64c-324bddcdfc72", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8e088420-73c4-4220-8908-20e829ab4cdf", - "resource": { - "resourceType": "Encounter", - "id": "8e088420-73c4-4220-8908-20e829ab4cdf", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2018-04-28T19:52:39-07:00", - "end": "2018-04-28T20:07:39-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0f4f91b8-4b10-4f5f-b94c-aebb27dd4adf", - "resource": { - "resourceType": "Condition", - "id": "0f4f91b8-4b10-4f5f-b94c-aebb27dd4adf", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:8e088420-73c4-4220-8908-20e829ab4cdf" - }, - "dateRecorded": "2018-04-28", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2018-04-28T19:52:39-07:00", - "abatementDateTime": "2018-05-19T19:52:39-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:a63c2763-4797-47e1-bdd6-286ded36b473", - "resource": { - "resourceType": "Claim", - "id": "a63c2763-4797-47e1-bdd6-286ded36b473", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f", - "resource": { - "resourceType": "Encounter", - "id": "14d585fa-87b6-48e6-b152-2e7c7cf42e8f", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2018-12-21T18:52:39-08:00", - "end": "2018-12-21T19:07:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f0da06c3-4e82-4bad-b0ce-8cdd3a35fed5", - "resource": { - "resourceType": "Observation", - "id": "f0da06c3-4e82-4bad-b0ce-8cdd3a35fed5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8036198b-2419-48ba-9379-d3aad37b83b5", - "resource": { - "resourceType": "Observation", - "id": "8036198b-2419-48ba-9379-d3aad37b83b5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:424ac9cd-a043-4d18-b98d-7ada684410a3", - "resource": { - "resourceType": "Observation", - "id": "424ac9cd-a043-4d18-b98d-7ada684410a3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 82.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ab45213-4f0c-4ac3-aa3f-edf5a8609243", - "resource": { - "resourceType": "Observation", - "id": "9ab45213-4f0c-4ac3-aa3f-edf5a8609243", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9eefb9cf-7abf-4bba-be76-cda789699973", - "resource": { - "resourceType": "Observation", - "id": "9eefb9cf-7abf-4bba-be76-cda789699973", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 88, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c50c7b2b-2dc1-4257-8b73-5ddfc7e431c8", - "resource": { - "resourceType": "Observation", - "id": "c50c7b2b-2dc1-4257-8b73-5ddfc7e431c8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 77, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d368f28e-dd42-48b1-b627-655459ca34b0", - "resource": { - "resourceType": "Observation", - "id": "d368f28e-dd42-48b1-b627-655459ca34b0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:286c0b51-4e55-4f09-8d88-5e477f08a78e", - "resource": { - "resourceType": "Observation", - "id": "286c0b51-4e55-4f09-8d88-5e477f08a78e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 74.830, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c3f782c5-1b4b-4cad-988b-3281a6c3e458", - "resource": { - "resourceType": "Observation", - "id": "c3f782c5-1b4b-4cad-988b-3281a6c3e458", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 10.580, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:597cc2e1-27e3-43f7-a4d1-7fd6ed2f1e59", - "resource": { - "resourceType": "Observation", - "id": "597cc2e1-27e3-43f7-a4d1-7fd6ed2f1e59", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 0.73000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e7b0f59c-0e17-4ada-bc9f-24f795399999", - "resource": { - "resourceType": "Observation", - "id": "e7b0f59c-0e17-4ada-bc9f-24f795399999", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 8.7000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d669964f-aad9-4d32-ac0e-24f939ee1bea", - "resource": { - "resourceType": "Observation", - "id": "d669964f-aad9-4d32-ac0e-24f939ee1bea", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 141.81, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a67392a6-a9c2-43ed-980d-000c20ce2bb8", - "resource": { - "resourceType": "Observation", - "id": "a67392a6-a9c2-43ed-980d-000c20ce2bb8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 4.8000, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d19c5563-b910-4f11-a909-dcfff2fa0328", - "resource": { - "resourceType": "Observation", - "id": "d19c5563-b910-4f11-a909-dcfff2fa0328", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 104.07, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", - "resource": { - "resourceType": "Observation", - "id": "85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 25.740, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5414a56b-7478-4e39-800f-42addf185b68", - "resource": { - "resourceType": "Observation", - "id": "5414a56b-7478-4e39-800f-42addf185b68", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 176.96, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2f58954-2a4c-4545-b908-42cd1f29eed6", - "resource": { - "resourceType": "Observation", - "id": "a2f58954-2a4c-4545-b908-42cd1f29eed6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 115.38, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:492c726d-55f5-4387-b3b7-91e7de7c9456", - "resource": { - "resourceType": "Observation", - "id": "492c726d-55f5-4387-b3b7-91e7de7c9456", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 84.670, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:abc0ddcb-9d31-4fec-b134-319d4aca0760", - "resource": { - "resourceType": "Observation", - "id": "abc0ddcb-9d31-4fec-b134-319d4aca0760", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 69.210, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d40a0cd-355d-4295-9767-c856c2a3671e", - "resource": { - "resourceType": "Observation", - "id": "6d40a0cd-355d-4295-9767-c856c2a3671e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8fb75f2e-6374-4b3b-9b84-8e0080e1a865", - "resource": { - "resourceType": "Observation", - "id": "8fb75f2e-6374-4b3b-9b84-8e0080e1a865", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "valueQuantity": { - "value": 6.2200, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f2046a66-34a7-49fa-955b-8766220f9c33", - "resource": { - "resourceType": "Immunization", - "id": "f2046a66-34a7-49fa-955b-8766220f9c33", - "status": "completed", - "date": "2018-12-21T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:44250cc7-1dcb-4719-a661-5ad3ecc39e9d", - "resource": { - "resourceType": "Immunization", - "id": "44250cc7-1dcb-4719-a661-5ad3ecc39e9d", - "status": "completed", - "date": "2018-12-21T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:76f31666-ead0-46a3-88d3-9cf48214ac39", - "resource": { - "resourceType": "DiagnosticReport", - "id": "76f31666-ead0-46a3-88d3-9cf48214ac39", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:85aa4118-8ae6-46eb-9c8d-9f18192b9b7c", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0c8e6e91-0bc4-4a48-9af0-d5f3ee417b47", - "resource": { - "resourceType": "DiagnosticReport", - "id": "0c8e6e91-0bc4-4a48-9af0-d5f3ee417b47", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:14d585fa-87b6-48e6-b152-2e7c7cf42e8f" - }, - "effectiveDateTime": "2018-12-21T18:52:39-08:00", - "issued": "2018-12-21T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:abc0ddcb-9d31-4fec-b134-319d4aca0760", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:c484f502-22f3-40d3-b9f9-d095bc9834c7", - "resource": { - "resourceType": "Claim", - "id": "c484f502-22f3-40d3-b9f9-d095bc9834c7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406", - "resource": { - "resourceType": "Encounter", - "id": "de6ef63c-f784-41ef-ac59-c469e5469406", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b" - } - } - ], - "period": { - "start": "2019-12-27T18:52:39-08:00", - "end": "2019-12-27T19:22:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e844227d-02ac-4c76-bf53-5798afe9282b", - "resource": { - "resourceType": "Observation", - "id": "e844227d-02ac-4c76-bf53-5798afe9282b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 163.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:edf3da83-1aea-4d79-8c68-d2b0c271b66e", - "resource": { - "resourceType": "Observation", - "id": "edf3da83-1aea-4d79-8c68-d2b0c271b66e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f8ac30f9-2105-4db5-9829-48cc6dff5746", - "resource": { - "resourceType": "Observation", - "id": "f8ac30f9-2105-4db5-9829-48cc6dff5746", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 82, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25609f0d-d738-452b-8591-5df0c13df4bc", - "resource": { - "resourceType": "Observation", - "id": "25609f0d-d738-452b-8591-5df0c13df4bc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 30.540, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51645096-b906-4001-b323-f74ea0363d2e", - "resource": { - "resourceType": "Observation", - "id": "51645096-b906-4001-b323-f74ea0363d2e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 102, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6b0b9b0-6a11-455b-a355-45e4ddccd517", - "resource": { - "resourceType": "Observation", - "id": "e6b0b9b0-6a11-455b-a355-45e4ddccd517", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e6fb00c-99fe-4cfb-b520-9f51100ce8a2", - "resource": { - "resourceType": "Observation", - "id": "4e6fb00c-99fe-4cfb-b520-9f51100ce8a2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:225841e0-8739-4d9e-a95b-0ab0f6e747a0", - "resource": { - "resourceType": "Observation", - "id": "225841e0-8739-4d9e-a95b-0ab0f6e747a0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 75.830, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d8bacb0-a294-4871-8946-6788b0baa7f6", - "resource": { - "resourceType": "Observation", - "id": "7d8bacb0-a294-4871-8946-6788b0baa7f6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 14.080, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfb6035e-6bbd-4f78-bc25-2b938e082ed7", - "resource": { - "resourceType": "Observation", - "id": "cfb6035e-6bbd-4f78-bc25-2b938e082ed7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 1.2200, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ccc75f42-5132-4251-b4bc-549497813436", - "resource": { - "resourceType": "Observation", - "id": "ccc75f42-5132-4251-b4bc-549497813436", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 9.8800, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:197848ba-270c-4424-822c-a8d9df1b6390", - "resource": { - "resourceType": "Observation", - "id": "197848ba-270c-4424-822c-a8d9df1b6390", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 136.44, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b8ee620-c378-4c26-acf0-11852056a021", - "resource": { - "resourceType": "Observation", - "id": "2b8ee620-c378-4c26-acf0-11852056a021", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 5.0800, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4646236f-3f86-4d6b-b9a2-7976dcd935f4", - "resource": { - "resourceType": "Observation", - "id": "4646236f-3f86-4d6b-b9a2-7976dcd935f4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 102.30, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bb1daea-75c1-4efe-93ca-eb2be16dfc35", - "resource": { - "resourceType": "Observation", - "id": "7bb1daea-75c1-4efe-93ca-eb2be16dfc35", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 21.710, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aa9a08a6-22eb-4685-9d7b-e91938c2b905", - "resource": { - "resourceType": "Observation", - "id": "aa9a08a6-22eb-4685-9d7b-e91938c2b905", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bec48bed-ac4f-4edd-aad8-8241711df36d", - "resource": { - "resourceType": "Observation", - "id": "bec48bed-ac4f-4edd-aad8-8241711df36d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "valueQuantity": { - "value": 5.8200, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1311b122-aed7-4ebc-bfa1-fc123ba3c6e4", - "resource": { - "resourceType": "Procedure", - "id": "1311b122-aed7-4ebc-bfa1-fc123ba3c6e4", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2019-12-27T18:52:39-08:00", - "end": "2019-12-27T19:07:39-08:00" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0cd4e24a-831a-4cb3-9b1c-2f6437e5b7ec", - "resource": { - "resourceType": "Immunization", - "id": "0cd4e24a-831a-4cb3-9b1c-2f6437e5b7ec", - "status": "completed", - "date": "2019-12-27T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d3eaf31a-f32c-4550-8acb-75312b86e276", - "resource": { - "resourceType": "Immunization", - "id": "d3eaf31a-f32c-4550-8acb-75312b86e276", - "status": "completed", - "date": "2019-12-27T18:52:39-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "33", - "display": "pneumococcal polysaccharide vaccine, 23 valent" - } - ], - "text": "pneumococcal polysaccharide vaccine, 23 valent" - }, - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b267580e-6b7e-41e2-9fd2-c3de5312fd7a", - "resource": { - "resourceType": "DiagnosticReport", - "id": "b267580e-6b7e-41e2-9fd2-c3de5312fd7a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "encounter": { - "reference": "urn:uuid:de6ef63c-f784-41ef-ac59-c469e5469406" - }, - "effectiveDateTime": "2019-12-27T18:52:39-08:00", - "issued": "2019-12-27T18:52:39.493-08:00", - "performer": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "result": [ - { - "reference": "urn:uuid:7bb1daea-75c1-4efe-93ca-eb2be16dfc35", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:ce3a8d7b-9307-4e1b-8eb3-1f887c0c80e4", - "resource": { - "resourceType": "Claim", - "id": "ce3a8d7b-9307-4e1b-8eb3-1f887c0c80e4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "33", - "display": "pneumococcal polysaccharide vaccine, 23 valent" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 702.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3a1dd113-8872-4a7e-b9c0-f8d2f824a81b", - "resource": { - "resourceType": "Encounter", - "id": "3a1dd113-8872-4a7e-b9c0-f8d2f824a81b", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d" - } - } - ], - "period": { - "start": "2020-01-17T18:52:39-08:00", - "end": "2020-01-17T19:39:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3509b8dc-6f14-4290-a891-e3820575f275", - "resource": { - "resourceType": "Procedure", - "id": "3509b8dc-6f14-4290-a891-e3820575f275", - "subject": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "performedPeriod": { - "start": "2020-01-17T18:52:39-08:00", - "end": "2020-01-17T19:24:39-08:00" - }, - "encounter": { - "reference": "urn:uuid:3a1dd113-8872-4a7e-b9c0-f8d2f824a81b" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c5b67198-55de-4324-b547-1e6c2dbe40b1", - "resource": { - "resourceType": "Claim", - "id": "c5b67198-55de-4324-b547-1e6c2dbe40b1", - "type": "institutional", - "organization": { - "reference": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:ed7a9f5c-37ea-4767-95ba-a6783e9500b3" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - }, - "net": { - "value": 11482.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json deleted file mode 100644 index 0acb308d86d3..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Elma843_Hoppe518_1fd2683f-2a56-47c7-b674-98f2cc8319e7.json +++ /dev/null @@ -1,10877 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7", - "resource": { - "resourceType": "Patient", - "id": "1fd2683f-2a56-47c7-b674-98f2cc8319e7", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 481861502406456147 Population seed: 1586309754086
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Race", - "code": "2106-3", - "display": "White" - } - ], - "text": "White" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Ethnicity", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - ], - "text": "Not Hispanic or Latino" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Dann525 Davis923" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Northampton", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.08312407189384233 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 43.91687592810616 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR" - } - ] - }, - "system": "http://hospital.smarthealthit.org", - "value": "1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB" - } - ] - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-31-6576" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "DL" - } - ] - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99995564" - } - ], - "name": [ - { - "use": "official", - "family": [ - "Hoppe518" - ], - "given": [ - "Elma843" - ], - "prefix": [ - "Mrs." - ] - }, - { - "use": "maiden", - "family": [ - "Luettgen772" - ], - "given": [ - "Elma843" - ], - "prefix": [ - "Mrs." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-460-2745", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1975-07-18", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.51197534529821 - }, - { - "url": "longitude", - "valueDecimal": -70.93304585681999 - } - ] - } - ], - "line": [ - "636 Brakus Station Unit 87" - ], - "city": "Salem", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "M" - } - ] - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9", - "resource": { - "resourceType": "Organization", - "id": "4861d01f-019c-3dac-a153-8334e50919f9", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "4861d01f-019c-3dac-a153-8334e50919f9" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "NORTH SHORE MEDICAL CENTER -", - "telecom": [ - { - "system": "phone", - "value": "9787411215" - } - ], - "address": [ - { - "line": [ - "81 HIGHLAND AVENUE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e", - "resource": { - "resourceType": "Practitioner", - "id": "b102f703-9e8c-37a2-8eea-08b67c680a1e", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "160" - } - ], - "active": true, - "name": { - "family": [ - "Roob72" - ], - "given": [ - "Isobel140" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "81 HIGHLAND AVENUE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:d33a3c86-5107-4f25-9130-bd7c7f7f0584", - "resource": { - "resourceType": "Encounter", - "id": "d33a3c86-5107-4f25-9130-bd7c7f7f0584", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "1990-08-30T12:47:52-07:00", - "end": "1990-08-30T13:02:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "36971009", - "display": "Sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e9c54b79-9861-4d16-8417-31c421b85020", - "resource": { - "resourceType": "Condition", - "id": "e9c54b79-9861-4d16-8417-31c421b85020", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:d33a3c86-5107-4f25-9130-bd7c7f7f0584" - }, - "dateRecorded": "1990-08-30", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "40055000", - "display": "Chronic sinusitis (disorder)" - } - ], - "text": "Chronic sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "1990-08-30T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:540c7750-af84-4144-921e-81bdea3c82ae", - "resource": { - "resourceType": "Claim", - "id": "540c7750-af84-4144-921e-81bdea3c82ae", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "40055000", - "display": "Chronic sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1eefc783-154d-4301-bdaf-53ec6c0ad698", - "resource": { - "resourceType": "Encounter", - "id": "1eefc783-154d-4301-bdaf-53ec6c0ad698", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2009-07-15T12:47:52-07:00", - "end": "2009-07-15T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ec829d7d-2c8f-4efc-ae24-968a9bfbb25c", - "resource": { - "resourceType": "MedicationOrder", - "id": "ec829d7d-2c8f-4efc-ae24-968a9bfbb25c", - "dateWritten": "2009-07-15T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:1eefc783-154d-4301-bdaf-53ec6c0ad698" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "807283", - "display": "Mirena 52 MG Intrauterine System" - } - ], - "text": "Mirena 52 MG Intrauterine System" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:5330f702-b006-40b9-86c7-e0d21dafda85", - "resource": { - "resourceType": "Claim", - "id": "5330f702-b006-40b9-86c7-e0d21dafda85", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:ec829d7d-2c8f-4efc-ae24-968a9bfbb25c" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a38ec00e-36e1-4c1a-849f-2db5e3b51f2b", - "resource": { - "resourceType": "Claim", - "id": "a38ec00e-36e1-4c1a-849f-2db5e3b51f2b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:24ed126d-6cd7-4ab5-af33-940eef487e7d", - "resource": { - "resourceType": "Encounter", - "id": "24ed126d-6cd7-4ab5-af33-940eef487e7d", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2010-04-30T12:47:52-07:00", - "end": "2010-04-30T13:13:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cf6bcedf-7d2f-47cf-a176-10570dcd40ee", - "resource": { - "resourceType": "Procedure", - "id": "cf6bcedf-7d2f-47cf-a176-10570dcd40ee", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2010-04-30T12:47:52-07:00", - "end": "2010-04-30T12:58:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:24ed126d-6cd7-4ab5-af33-940eef487e7d" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:238439bd-9910-4b3d-99ed-97e680b8693c", - "resource": { - "resourceType": "Claim", - "id": "238439bd-9910-4b3d-99ed-97e680b8693c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 4861.95, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6e6753f7-a24b-49e2-a77a-45de4bc3e3b6", - "resource": { - "resourceType": "Encounter", - "id": "6e6753f7-a24b-49e2-a77a-45de4bc3e3b6", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2010-07-30T12:47:52-07:00", - "end": "2010-07-30T13:21:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:341a008e-a687-4832-a9f8-d06b690f856a", - "resource": { - "resourceType": "Procedure", - "id": "341a008e-a687-4832-a9f8-d06b690f856a", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2010-07-30T12:47:52-07:00", - "end": "2010-07-30T13:06:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:6e6753f7-a24b-49e2-a77a-45de4bc3e3b6" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2e1406b2-2b26-4f34-8f4c-27723549b121", - "resource": { - "resourceType": "Claim", - "id": "2e1406b2-2b26-4f34-8f4c-27723549b121", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 4073.77, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:866c5085-52c9-4ecf-b791-b0e87e392ffc", - "resource": { - "resourceType": "Encounter", - "id": "866c5085-52c9-4ecf-b791-b0e87e392ffc", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2010-10-29T12:47:52-07:00", - "end": "2010-10-29T13:13:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3f3ff32b-2f53-42bd-a9b1-05ad5ee9297c", - "resource": { - "resourceType": "Procedure", - "id": "3f3ff32b-2f53-42bd-a9b1-05ad5ee9297c", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2010-10-29T12:47:52-07:00", - "end": "2010-10-29T12:58:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:866c5085-52c9-4ecf-b791-b0e87e392ffc" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4b77465b-b3ad-4b80-959d-feb3dcb92d99", - "resource": { - "resourceType": "Claim", - "id": "4b77465b-b3ad-4b80-959d-feb3dcb92d99", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 2455.95, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c34dd6dc-55e6-4246-a054-1d6deeb78195", - "resource": { - "resourceType": "Encounter", - "id": "c34dd6dc-55e6-4246-a054-1d6deeb78195", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2011-01-28T11:47:52-08:00", - "end": "2011-01-28T12:16:52-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:88ccca2b-124d-49d0-906f-c836c7f60d94", - "resource": { - "resourceType": "Procedure", - "id": "88ccca2b-124d-49d0-906f-c836c7f60d94", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2011-01-28T11:47:52-08:00", - "end": "2011-01-28T12:01:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:c34dd6dc-55e6-4246-a054-1d6deeb78195" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:eac33817-4b7b-44f1-b3ed-56d215e51fb9", - "resource": { - "resourceType": "Claim", - "id": "eac33817-4b7b-44f1-b3ed-56d215e51fb9", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 2429.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:605546ae-6d8a-4aa7-9f7d-6eba37402f65", - "resource": { - "resourceType": "Encounter", - "id": "605546ae-6d8a-4aa7-9f7d-6eba37402f65", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2011-04-29T12:47:52-07:00", - "end": "2011-04-29T13:17:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:319052d5-5d6a-408e-876b-a8a9bbcf7406", - "resource": { - "resourceType": "Procedure", - "id": "319052d5-5d6a-408e-876b-a8a9bbcf7406", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2011-04-29T12:47:52-07:00", - "end": "2011-04-29T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:605546ae-6d8a-4aa7-9f7d-6eba37402f65" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7e1060d8-1648-443d-957a-9255f6c36db0", - "resource": { - "resourceType": "Claim", - "id": "7e1060d8-1648-443d-957a-9255f6c36db0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 4145.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2f67e2fe-1b41-49e4-8a76-126cc0272375", - "resource": { - "resourceType": "Encounter", - "id": "2f67e2fe-1b41-49e4-8a76-126cc0272375", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2011-07-29T12:47:52-07:00", - "end": "2011-07-29T13:18:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:61bb7894-a14f-43ef-92eb-15cc3f8f1e6e", - "resource": { - "resourceType": "Procedure", - "id": "61bb7894-a14f-43ef-92eb-15cc3f8f1e6e", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2011-07-29T12:47:52-07:00", - "end": "2011-07-29T13:03:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:2f67e2fe-1b41-49e4-8a76-126cc0272375" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5f5d52dd-d0ba-4934-8309-79c2b9cac398", - "resource": { - "resourceType": "Claim", - "id": "5f5d52dd-d0ba-4934-8309-79c2b9cac398", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 2891.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff", - "resource": { - "resourceType": "Encounter", - "id": "b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2011-10-28T12:47:52-07:00", - "end": "2011-10-28T13:19:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4c5adff2-56ec-45e0-a95f-c8e30a02f8a1", - "resource": { - "resourceType": "Procedure", - "id": "4c5adff2-56ec-45e0-a95f-c8e30a02f8a1", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2011-10-28T12:47:52-07:00", - "end": "2011-10-28T13:04:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:b87b91f3-eb39-4bc4-84fe-e69d0ba1e9ff" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d402dec1-770c-4e88-9711-99db92a624cc", - "resource": { - "resourceType": "Claim", - "id": "d402dec1-770c-4e88-9711-99db92a624cc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 3754.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6b944acc-27cb-443c-885e-1176f99fc33d", - "resource": { - "resourceType": "Encounter", - "id": "6b944acc-27cb-443c-885e-1176f99fc33d", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-01-27T11:47:52-08:00", - "end": "2012-01-27T12:17:52-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6cd3de1b-f110-4446-a70c-83daae4a60e8", - "resource": { - "resourceType": "Procedure", - "id": "6cd3de1b-f110-4446-a70c-83daae4a60e8", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2012-01-27T11:47:52-08:00", - "end": "2012-01-27T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:6b944acc-27cb-443c-885e-1176f99fc33d" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:98b5c06e-96a0-4049-858b-7dacb359149d", - "resource": { - "resourceType": "Claim", - "id": "98b5c06e-96a0-4049-858b-7dacb359149d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 4855.31, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:debb7550-af72-4a4c-8fed-99adeb9888e7", - "resource": { - "resourceType": "Encounter", - "id": "debb7550-af72-4a4c-8fed-99adeb9888e7", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-04-27T12:47:52-07:00", - "end": "2012-04-27T13:14:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:53183014-7229-4db9-9727-2d8ff21a6458", - "resource": { - "resourceType": "Procedure", - "id": "53183014-7229-4db9-9727-2d8ff21a6458", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2012-04-27T12:47:52-07:00", - "end": "2012-04-27T12:59:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:debb7550-af72-4a4c-8fed-99adeb9888e7" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:70e55020-84fd-4cc9-acc5-c72b8c16057d", - "resource": { - "resourceType": "Claim", - "id": "70e55020-84fd-4cc9-acc5-c72b8c16057d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 4120.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325", - "resource": { - "resourceType": "Encounter", - "id": "93bcef01-906b-461b-8f78-0415c0508325", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-06-08T12:47:52-07:00", - "end": "2012-06-08T13:02:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e8d7f17e-8a02-42d9-9b4e-423678966a60", - "resource": { - "resourceType": "Condition", - "id": "e8d7f17e-8a02-42d9-9b4e-423678966a60", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325" - }, - "dateRecorded": "2012-06-08", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2012-06-08T12:47:52-07:00", - "abatementDateTime": "2012-06-15T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:614b3cbd-915e-41a8-8a65-204d06f0046b", - "resource": { - "resourceType": "MedicationOrder", - "id": "614b3cbd-915e-41a8-8a65-204d06f0046b", - "dateWritten": "2012-06-08T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:93bcef01-906b-461b-8f78-0415c0508325" - }, - "reasonReference": { - "reference": "urn:uuid:e8d7f17e-8a02-42d9-9b4e-423678966a60" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "562251", - "display": "Amoxicillin 250 MG / Clavulanate 125 MG Oral Tablet" - } - ], - "text": "Amoxicillin 250 MG / Clavulanate 125 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:00d1a2b2-b430-4748-aa19-e66ad8cd3130", - "resource": { - "resourceType": "Claim", - "id": "00d1a2b2-b430-4748-aa19-e66ad8cd3130", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:614b3cbd-915e-41a8-8a65-204d06f0046b" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ac3c1599-fa11-400b-9443-cf809263a505", - "resource": { - "resourceType": "Claim", - "id": "ac3c1599-fa11-400b-9443-cf809263a505", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cdac7ae3-4d0a-498d-bf25-a77924b07742", - "resource": { - "resourceType": "Encounter", - "id": "cdac7ae3-4d0a-498d-bf25-a77924b07742", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-07-27T12:47:52-07:00", - "end": "2012-07-27T13:17:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9654efc0-fb9f-4a8a-8ce9-eace3654387e", - "resource": { - "resourceType": "Procedure", - "id": "9654efc0-fb9f-4a8a-8ce9-eace3654387e", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "performedPeriod": { - "start": "2012-07-27T12:47:52-07:00", - "end": "2012-07-27T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cdac7ae3-4d0a-498d-bf25-a77924b07742" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:6c746559-8ab3-45e8-ac21-28bd0856532d", - "resource": { - "resourceType": "Claim", - "id": "6c746559-8ab3-45e8-ac21-28bd0856532d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - }, - "net": { - "value": 1492.94, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e", - "resource": { - "resourceType": "Organization", - "id": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "COMMONWEALTH RADIOLOGY ASSOCIATES, INC", - "telecom": [ - { - "system": "phone", - "value": "978-741-1200" - } - ], - "address": [ - { - "line": [ - "81 HIGHLAND AVE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970-2714", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc", - "resource": { - "resourceType": "Practitioner", - "id": "f4f50d8e-b084-3bc2-a787-30f3a29983fc", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "78430" - } - ], - "active": true, - "name": { - "family": [ - "Keeling57" - ], - "given": [ - "Johana303" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "81 HIGHLAND AVE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970-2714", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf", - "resource": { - "resourceType": "Encounter", - "id": "cd0dbd40-f321-4b6c-92ce-054824f3deaf", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" - } - } - ], - "period": { - "start": "2012-10-05T12:47:52-07:00", - "end": "2012-10-05T13:17:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:08f686c7-d801-4c96-8b21-3423f4731705", - "resource": { - "resourceType": "Condition", - "id": "08f686c7-d801-4c96-8b21-3423f4731705", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "dateRecorded": "2012-10-05", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2012-10-05T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:d36eb33f-bdf3-4e43-b942-365fd45e2d72", - "resource": { - "resourceType": "Observation", - "id": "d36eb33f-bdf3-4e43-b942-365fd45e2d72", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 162.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:498b2f72-3f65-4cb5-b8b7-98b9ddc8cfb2", - "resource": { - "resourceType": "Observation", - "id": "498b2f72-3f65-4cb5-b8b7-98b9ddc8cfb2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7a3a94e0-8ec3-4149-88d7-86d593abfc37", - "resource": { - "resourceType": "Observation", - "id": "7a3a94e0-8ec3-4149-88d7-86d593abfc37", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 79.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:333526be-540d-4ff3-97ac-eaf9877875cc", - "resource": { - "resourceType": "Observation", - "id": "333526be-540d-4ff3-97ac-eaf9877875cc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 30.030, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:18aa97e8-7ef3-476d-a32b-b1476b27c612", - "resource": { - "resourceType": "Observation", - "id": "18aa97e8-7ef3-476d-a32b-b1476b27c612", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e38509ed-a272-4d9b-9519-77721a420a30", - "resource": { - "resourceType": "Observation", - "id": "e38509ed-a272-4d9b-9519-77721a420a30", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:521672ec-d76c-4276-9207-543c7b24446f", - "resource": { - "resourceType": "Observation", - "id": "521672ec-d76c-4276-9207-543c7b24446f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a08f9938-f74e-4116-9a13-2abb5d785639", - "resource": { - "resourceType": "Observation", - "id": "a08f9938-f74e-4116-9a13-2abb5d785639", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 173.54, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a985217-5c0a-429a-b3d8-86207a1f86f2", - "resource": { - "resourceType": "Observation", - "id": "3a985217-5c0a-429a-b3d8-86207a1f86f2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 123.96, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6750492-22bc-40c5-a5da-6794e3139738", - "resource": { - "resourceType": "Observation", - "id": "e6750492-22bc-40c5-a5da-6794e3139738", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 70.350, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7397bc8d-08db-43dc-a30a-14f21e7cb13d", - "resource": { - "resourceType": "Observation", - "id": "7397bc8d-08db-43dc-a30a-14f21e7cb13d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueQuantity": { - "value": 78.400, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:826af47f-b98d-481e-88ee-7a375c21792a", - "resource": { - "resourceType": "Observation", - "id": "826af47f-b98d-481e-88ee-7a375c21792a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8240e5f3-9962-41f9-a9aa-db6b3254bffe", - "resource": { - "resourceType": "Procedure", - "id": "8240e5f3-9962-41f9-a9aa-db6b3254bffe", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2012-10-05T12:47:52-07:00", - "end": "2012-10-05T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:273b425d-2435-433c-b2dc-baeecc4498c4", - "resource": { - "resourceType": "Immunization", - "id": "273b425d-2435-433c-b2dc-baeecc4498c4", - "status": "completed", - "date": "2012-10-05T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:9cf84800-67b1-4016-aa14-cc4fb5aad760", - "resource": { - "resourceType": "DiagnosticReport", - "id": "9cf84800-67b1-4016-aa14-cc4fb5aad760", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cd0dbd40-f321-4b6c-92ce-054824f3deaf" - }, - "effectiveDateTime": "2012-10-05T12:47:52-07:00", - "issued": "2012-10-05T12:47:52.730-07:00", - "performer": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "result": [ - { - "reference": "urn:uuid:7397bc8d-08db-43dc-a30a-14f21e7cb13d", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:3d2ad77e-b88d-457a-9574-de0d4ec5e25c", - "resource": { - "resourceType": "Claim", - "id": "3d2ad77e-b88d-457a-9574-de0d4ec5e25c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 1007.40, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8", - "resource": { - "resourceType": "Encounter", - "id": "d0e4fada-dc01-4853-8bd2-f8c45e6455a8", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T18:02:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f", - "resource": { - "resourceType": "Condition", - "id": "72e7d27a-a498-407c-afb8-6fd1b50b1a8f", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - }, - "dateRecorded": "2012-10-19", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ], - "text": "Normal pregnancy" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2012-10-19T12:47:52-07:00", - "abatementDateTime": "2013-05-17T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ee3fd7c0-f1c7-4f07-9e93-46eb5a4e99ff", - "resource": { - "resourceType": "Procedure", - "id": "ee3fd7c0-f1c7-4f07-9e93-46eb5a4e99ff", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "252160004", - "display": "Standard pregnancy test" - } - ], - "text": "Standard pregnancy test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:05cf5d86-32cb-4e39-9b98-ceeeba210eed", - "resource": { - "resourceType": "Procedure", - "id": "05cf5d86-32cb-4e39-9b98-ceeeba210eed", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "169230002", - "display": "Ultrasound scan for fetal viability" - } - ], - "text": "Ultrasound scan for fetal viability" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:98e52fb8-ad50-4459-b782-d479e050cab5", - "resource": { - "resourceType": "Procedure", - "id": "98e52fb8-ad50-4459-b782-d479e050cab5", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:38ef00c7-d111-4537-9c0b-772f25b91941", - "resource": { - "resourceType": "Procedure", - "id": "38ef00c7-d111-4537-9c0b-772f25b91941", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:8089eaea-d373-4c02-af87-794a39058207", - "resource": { - "resourceType": "Procedure", - "id": "8089eaea-d373-4c02-af87-794a39058207", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44608003", - "display": "Blood typing, RH typing" - } - ], - "text": "Blood typing, RH typing" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c96c9f12-6f03-462a-b03c-ef317e417d79", - "resource": { - "resourceType": "Procedure", - "id": "c96c9f12-6f03-462a-b03c-ef317e417d79", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104091002", - "display": "Hemoglobin / Hematocrit / Platelet count" - } - ], - "text": "Hemoglobin / Hematocrit / Platelet count" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:425f4eef-68f8-4d75-a899-21b09fe0ad19", - "resource": { - "resourceType": "Procedure", - "id": "425f4eef-68f8-4d75-a899-21b09fe0ad19", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "47758006", - "display": "Hepatitis B Surface Antigen Measurement" - } - ], - "text": "Hepatitis B Surface Antigen Measurement" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f68721ca-4410-4e4f-87d5-b2424ef5b129", - "resource": { - "resourceType": "Procedure", - "id": "f68721ca-4410-4e4f-87d5-b2424ef5b129", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "31676001", - "display": "Human immunodeficiency virus antigen test" - } - ], - "text": "Human immunodeficiency virus antigen test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2446e543-84b3-48e5-829f-a3ff9f5e3d98", - "resource": { - "resourceType": "Procedure", - "id": "2446e543-84b3-48e5-829f-a3ff9f5e3d98", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "310861008", - "display": "Chlamydia antigen test" - } - ], - "text": "Chlamydia antigen test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b6628389-c214-4890-88e6-7f15a3701212", - "resource": { - "resourceType": "Procedure", - "id": "b6628389-c214-4890-88e6-7f15a3701212", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "165829005", - "display": "Gonorrhea infection test" - } - ], - "text": "Gonorrhea infection test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:eaf71f03-b202-4f92-a7ce-70c9154f64cc", - "resource": { - "resourceType": "Procedure", - "id": "eaf71f03-b202-4f92-a7ce-70c9154f64cc", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269828009", - "display": "Syphilis infection test" - } - ], - "text": "Syphilis infection test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b7f227b6-4d16-4148-96e3-dac95b399c84", - "resource": { - "resourceType": "Procedure", - "id": "b7f227b6-4d16-4148-96e3-dac95b399c84", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117010004", - "display": "Urine culture" - } - ], - "text": "Urine culture" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:30ec60ab-4285-4784-bfb8-c60c552edd91", - "resource": { - "resourceType": "Procedure", - "id": "30ec60ab-4285-4784-bfb8-c60c552edd91", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "90226004", - "display": "Cytopathology procedure, preparation of smear, genital source" - } - ], - "text": "Cytopathology procedure, preparation of smear, genital source" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0e0acfbd-8e13-4068-be5b-c8db28aa8370", - "resource": { - "resourceType": "Procedure", - "id": "0e0acfbd-8e13-4068-be5b-c8db28aa8370", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "395123002", - "display": "Urine screening test for diabetes" - } - ], - "text": "Urine screening test for diabetes" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4befc1a0-2733-4e67-be8e-6e28f3607208", - "resource": { - "resourceType": "Procedure", - "id": "4befc1a0-2733-4e67-be8e-6e28f3607208", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104375008", - "display": "Hepatitis C antibody test" - } - ], - "text": "Hepatitis C antibody test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4fdf0764-5136-4bfc-b05b-4fc7eb43277f", - "resource": { - "resourceType": "Procedure", - "id": "4fdf0764-5136-4bfc-b05b-4fc7eb43277f", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "169690007", - "display": "Rubella screening" - } - ], - "text": "Rubella screening" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:97ea62c5-f862-4e24-8f49-6d856bade41b", - "resource": { - "resourceType": "Procedure", - "id": "97ea62c5-f862-4e24-8f49-6d856bade41b", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104326007", - "display": "Measurement of Varicella-zoster virus antibody" - } - ], - "text": "Measurement of Varicella-zoster virus antibody" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0d6f1573-9c15-46bc-a3be-75016463add0", - "resource": { - "resourceType": "Procedure", - "id": "0d6f1573-9c15-46bc-a3be-75016463add0", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "28163009", - "display": "Skin test for tuberculosis" - } - ], - "text": "Skin test for tuberculosis" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b3cced7a-ef97-4147-b87d-1b9d79d30142", - "resource": { - "resourceType": "Procedure", - "id": "b3cced7a-ef97-4147-b87d-1b9d79d30142", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "167271000", - "display": "Urine protein test" - } - ], - "text": "Urine protein test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:532d6177-9796-4068-9cda-f05c143bf578", - "resource": { - "resourceType": "Procedure", - "id": "532d6177-9796-4068-9cda-f05c143bf578", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination of mother" - } - ], - "text": "Physical examination of mother" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2012-10-19T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ac3a8152-5640-473d-8609-a6896c30f6c9", - "resource": { - "resourceType": "CarePlan", - "id": "ac3a8152-5640-473d-8609-a6896c30f6c9", - "text": { - "status": "generated", - "div": "
    Routine antenatal care
    " - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "context": { - "reference": "urn:uuid:d0e4fada-dc01-4853-8bd2-f8c45e6455a8" - }, - "period": { - "start": "2012-10-19T12:47:52-07:00", - "end": "2013-05-17T12:47:52-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "134435003", - "display": "Routine antenatal care" - } - ], - "text": "Routine antenatal care" - } - ], - "addresses": [ - { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "135892000", - "display": "Antenatal education" - } - ], - "text": "Antenatal education" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "713076009", - "display": "Antenatal risk assessment" - } - ], - "text": "Antenatal risk assessment" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312404004", - "display": "Antenatal blood tests" - } - ], - "text": "Antenatal blood tests" - }, - "status": "completed", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:5708f080-1539-4c2d-b6a4-fae148c86357", - "resource": { - "resourceType": "Claim", - "id": "5708f080-1539-4c2d-b6a4-fae148c86357", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "252160004", - "display": "Standard pregnancy test" - }, - "net": { - "value": 5212.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "169230002", - "display": "Ultrasound scan for fetal viability" - }, - "net": { - "value": 15228.25, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 9562.18, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 7670.03, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "44608003", - "display": "Blood typing, RH typing" - }, - "net": { - "value": 1389.71, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 8, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "104091002", - "display": "Hemoglobin / Hematocrit / Platelet count" - }, - "net": { - "value": 2108.07, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 9, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "47758006", - "display": "Hepatitis B Surface Antigen Measurement" - }, - "net": { - "value": 2914.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 10, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "31676001", - "display": "Human immunodeficiency virus antigen test" - }, - "net": { - "value": 2848.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 11, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "310861008", - "display": "Chlamydia antigen test" - }, - "net": { - "value": 2512.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 12, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "165829005", - "display": "Gonorrhea infection test" - }, - "net": { - "value": 2714.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 13, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "269828009", - "display": "Syphilis infection test" - }, - "net": { - "value": 2136.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 14, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "117010004", - "display": "Urine culture" - }, - "net": { - "value": 2853.03, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 15, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "90226004", - "display": "Cytopathology procedure, preparation of smear, genital source" - }, - "net": { - "value": 2717.85, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 16, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "395123002", - "display": "Urine screening test for diabetes" - }, - "net": { - "value": 2950.24, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 17, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "104375008", - "display": "Hepatitis C antibody test" - }, - "net": { - "value": 2176.72, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 18, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "169690007", - "display": "Rubella screening" - }, - "net": { - "value": 2229.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 19, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "104326007", - "display": "Measurement of Varicella-zoster virus antibody" - }, - "net": { - "value": 2864.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 20, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "28163009", - "display": "Skin test for tuberculosis" - }, - "net": { - "value": 1485.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 21, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "167271000", - "display": "Urine protein test" - }, - "net": { - "value": 2916.90, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 22, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination of mother" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4", - "resource": { - "resourceType": "Encounter", - "id": "4153dbe1-f877-43eb-8498-627b47dbabe4", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-11-16T11:47:52-08:00", - "end": "2012-11-16T12:47:52-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a2de3620-89d7-4f5e-83a9-43ef8ae2410a", - "resource": { - "resourceType": "Procedure", - "id": "a2de3620-89d7-4f5e-83a9-43ef8ae2410a", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-11-16T11:47:52-08:00", - "end": "2012-11-16T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4293be80-fa68-455f-9d46-c71ea4279544", - "resource": { - "resourceType": "Procedure", - "id": "4293be80-fa68-455f-9d46-c71ea4279544", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-11-16T11:47:52-08:00", - "end": "2012-11-16T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3b1d7021-63c6-4e72-bc8f-af7a6313989e", - "resource": { - "resourceType": "Procedure", - "id": "3b1d7021-63c6-4e72-bc8f-af7a6313989e", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "443529005", - "display": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" - } - ], - "text": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-11-16T11:47:52-08:00", - "end": "2012-11-16T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:4153dbe1-f877-43eb-8498-627b47dbabe4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b908fca2-c9b4-4dc7-beb8-d915d358e550", - "resource": { - "resourceType": "Claim", - "id": "b908fca2-c9b4-4dc7-beb8-d915d358e550", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 7348.06, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 9287.00, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "443529005", - "display": "Screening for chromosomal aneuploidy in prenatal amniotic fluid" - }, - "net": { - "value": 1654.54, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2", - "resource": { - "resourceType": "Encounter", - "id": "42034ba4-ed69-459c-b832-6c15bdefc7c2", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2012-12-14T11:47:52-08:00", - "end": "2012-12-14T13:02:52-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a90044b0-8810-4b11-b628-2501c650dd4d", - "resource": { - "resourceType": "Procedure", - "id": "a90044b0-8810-4b11-b628-2501c650dd4d", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271442007", - "display": "Fetal anatomy study" - } - ], - "text": "Fetal anatomy study" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-12-14T11:47:52-08:00", - "end": "2012-12-14T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a8534dc7-6e36-44f1-b7bd-32db0946e08f", - "resource": { - "resourceType": "Procedure", - "id": "a8534dc7-6e36-44f1-b7bd-32db0946e08f", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "275833003", - "display": "Alpha-fetoprotein test" - } - ], - "text": "Alpha-fetoprotein test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-12-14T11:47:52-08:00", - "end": "2012-12-14T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:17ea9dcb-cff1-49ff-af65-d0a2ba16ffcc", - "resource": { - "resourceType": "Procedure", - "id": "17ea9dcb-cff1-49ff-af65-d0a2ba16ffcc", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-12-14T11:47:52-08:00", - "end": "2012-12-14T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c0576791-0d92-4b94-a133-5f6e789c614d", - "resource": { - "resourceType": "Procedure", - "id": "c0576791-0d92-4b94-a133-5f6e789c614d", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2012-12-14T11:47:52-08:00", - "end": "2012-12-14T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:42034ba4-ed69-459c-b832-6c15bdefc7c2" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:344aff95-7921-4fa6-8595-79addeccc669", - "resource": { - "resourceType": "Claim", - "id": "344aff95-7921-4fa6-8595-79addeccc669", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "271442007", - "display": "Fetal anatomy study" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "275833003", - "display": "Alpha-fetoprotein test" - }, - "net": { - "value": 1929.30, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 6891.86, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 6656.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d", - "resource": { - "resourceType": "Encounter", - "id": "ed2fcb69-b3f9-4298-bfe2-e01c331cef8d", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-01-11T11:47:52-08:00", - "end": "2013-01-11T12:32:52-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:eb158c5e-0c6d-4748-a302-35eafa26752b", - "resource": { - "resourceType": "Procedure", - "id": "eb158c5e-0c6d-4748-a302-35eafa26752b", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-01-11T11:47:52-08:00", - "end": "2013-01-11T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:52b94e1a-141f-4930-be60-4c6361eb6108", - "resource": { - "resourceType": "Procedure", - "id": "52b94e1a-141f-4930-be60-4c6361eb6108", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-01-11T11:47:52-08:00", - "end": "2013-01-11T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:ed2fcb69-b3f9-4298-bfe2-e01c331cef8d" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:44e2a39e-0d74-4bdc-b577-3217f7db7550", - "resource": { - "resourceType": "Claim", - "id": "44e2a39e-0d74-4bdc-b577-3217f7db7550", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 7288.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 5248.44, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093", - "resource": { - "resourceType": "Encounter", - "id": "a40ac6e5-fef8-4875-a8d5-709953a76093", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-02-08T11:47:52-08:00", - "end": "2013-02-08T12:32:52-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d606b989-27ec-40eb-b6a2-eb23cb5281a9", - "resource": { - "resourceType": "Procedure", - "id": "d606b989-27ec-40eb-b6a2-eb23cb5281a9", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-02-08T11:47:52-08:00", - "end": "2013-02-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f6274adb-25b2-4de4-b36d-c30cc1e5a40c", - "resource": { - "resourceType": "Procedure", - "id": "f6274adb-25b2-4de4-b36d-c30cc1e5a40c", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-02-08T11:47:52-08:00", - "end": "2013-02-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:a40ac6e5-fef8-4875-a8d5-709953a76093" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b3ff90e4-9960-4694-aa63-6dab8fbac5f0", - "resource": { - "resourceType": "Claim", - "id": "b3ff90e4-9960-4694-aa63-6dab8fbac5f0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 6444.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 5708.97, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907", - "resource": { - "resourceType": "Encounter", - "id": "88d42111-285b-4199-8000-a10a199a9907", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T13:17:52-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3d506b4c-cd43-4d04-a874-0835cfc28358", - "resource": { - "resourceType": "Procedure", - "id": "3d506b4c-cd43-4d04-a874-0835cfc28358", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104091002", - "display": "Hemoglobin / Hematocrit / Platelet count" - } - ], - "text": "Hemoglobin / Hematocrit / Platelet count" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:eacdbd19-ba24-4dbf-93d4-016282aee0d1", - "resource": { - "resourceType": "Procedure", - "id": "eacdbd19-ba24-4dbf-93d4-016282aee0d1", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "399014008", - "display": "Vaccination for diphtheria, pertussis, and tetanus" - } - ], - "text": "Vaccination for diphtheria, pertussis, and tetanus" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:33b20d57-35ee-498f-b49a-cd3867b88747", - "resource": { - "resourceType": "Procedure", - "id": "33b20d57-35ee-498f-b49a-cd3867b88747", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "268556000", - "display": "Urine screening for glucose" - } - ], - "text": "Urine screening for glucose" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9f5d2bae-f114-4ebd-8316-bf9b713b8217", - "resource": { - "resourceType": "Procedure", - "id": "9f5d2bae-f114-4ebd-8316-bf9b713b8217", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3e565b4d-3353-43aa-a7b1-6e4e98af87b4", - "resource": { - "resourceType": "Procedure", - "id": "3e565b4d-3353-43aa-a7b1-6e4e98af87b4", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-03-08T11:47:52-08:00", - "end": "2013-03-08T12:02:52-08:00" - }, - "encounter": { - "reference": "urn:uuid:88d42111-285b-4199-8000-a10a199a9907" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d53524f0-b28e-45e4-a40a-20558b9cc51d", - "resource": { - "resourceType": "Claim", - "id": "d53524f0-b28e-45e4-a40a-20558b9cc51d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "104091002", - "display": "Hemoglobin / Hematocrit / Platelet count" - }, - "net": { - "value": 1837.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "399014008", - "display": "Vaccination for diphtheria, pertussis, and tetanus" - }, - "net": { - "value": 3667.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "268556000", - "display": "Urine screening for glucose" - }, - "net": { - "value": 3248.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 10305.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 7134.31, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428", - "resource": { - "resourceType": "Encounter", - "id": "5c17fe8f-881b-4071-a2b6-28a10ab25428", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-04-05T12:47:52-07:00", - "end": "2013-04-05T13:32:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:da4576b1-9b56-4168-8f83-7388dd6d3b06", - "resource": { - "resourceType": "Procedure", - "id": "da4576b1-9b56-4168-8f83-7388dd6d3b06", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-04-05T12:47:52-07:00", - "end": "2013-04-05T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e9d98b4d-e178-47c8-ba7f-ddfd3bcc1f4e", - "resource": { - "resourceType": "Procedure", - "id": "e9d98b4d-e178-47c8-ba7f-ddfd3bcc1f4e", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-04-05T12:47:52-07:00", - "end": "2013-04-05T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:5c17fe8f-881b-4071-a2b6-28a10ab25428" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:59505f7c-1995-41d7-88f1-7fec7f002971", - "resource": { - "resourceType": "Claim", - "id": "59505f7c-1995-41d7-88f1-7fec7f002971", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 7325.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 10230.59, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c", - "resource": { - "resourceType": "Encounter", - "id": "fd5881ab-1d21-4aca-b59c-08305ee9079c", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-05-03T12:47:52-07:00", - "end": "2013-05-03T13:47:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d73b252f-2b8b-4e50-9ff4-e8177f090151", - "resource": { - "resourceType": "Procedure", - "id": "d73b252f-2b8b-4e50-9ff4-e8177f090151", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "118001005", - "display": "Streptococcus pneumoniae group B antigen test" - } - ], - "text": "Streptococcus pneumoniae group B antigen test" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-05-03T12:47:52-07:00", - "end": "2013-05-03T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d4c98a0f-3057-4763-adca-33d1728e08cd", - "resource": { - "resourceType": "Procedure", - "id": "d4c98a0f-3057-4763-adca-33d1728e08cd", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - } - ], - "text": "Evaluation of uterine fundal height" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-05-03T12:47:52-07:00", - "end": "2013-05-03T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:dddbbe44-e447-46cc-8fa5-8c0872de32ee", - "resource": { - "resourceType": "Procedure", - "id": "dddbbe44-e447-46cc-8fa5-8c0872de32ee", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - } - ], - "text": "Auscultation of the fetal heart" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-05-03T12:47:52-07:00", - "end": "2013-05-03T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:fd5881ab-1d21-4aca-b59c-08305ee9079c" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:792a9ad3-a723-469a-a219-13495fb3b3a2", - "resource": { - "resourceType": "Claim", - "id": "792a9ad3-a723-469a-a219-13495fb3b3a2", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "118001005", - "display": "Streptococcus pneumoniae group B antigen test" - }, - "net": { - "value": 1207.97, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274804006", - "display": "Evaluation of uterine fundal height" - }, - "net": { - "value": 8244.47, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "225158009", - "display": "Auscultation of the fetal heart" - }, - "net": { - "value": 4228.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80", - "resource": { - "resourceType": "Encounter", - "id": "d51c12d4-eeab-4c80-b5bc-b1dc14235c80", - "status": "finished", - "class": "emergency", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "183460006", - "display": "Obstetric emergency hospital admission" - } - ], - "text": "Obstetric emergency hospital admission" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-05-17T12:47:52-07:00", - "end": "2013-05-17T14:17:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e44ef6b9-aa50-4296-93f7-021839e65728", - "resource": { - "resourceType": "Procedure", - "id": "e44ef6b9-aa50-4296-93f7-021839e65728", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "31208007", - "display": "Medical induction of labor" - } - ], - "text": "Medical induction of labor" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-05-17T12:47:52-07:00", - "end": "2013-05-17T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7f9d558f-e196-4c0c-b951-92ed7da439b3", - "resource": { - "resourceType": "Procedure", - "id": "7f9d558f-e196-4c0c-b951-92ed7da439b3", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "11466000", - "display": "Cesarean section" - } - ], - "text": "Cesarean section" - }, - "reasonReference": { - "reference": "urn:uuid:72e7d27a-a498-407c-afb8-6fd1b50b1a8f" - }, - "performedPeriod": { - "start": "2013-05-17T12:47:52-07:00", - "end": "2013-05-17T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:d51c12d4-eeab-4c80-b5bc-b1dc14235c80" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:924eb1bd-bf53-4118-a423-36b964bae627", - "resource": { - "resourceType": "Claim", - "id": "924eb1bd-bf53-4118-a423-36b964bae627", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "183460006", - "display": "Obstetric emergency hospital admission" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "31208007", - "display": "Medical induction of labor" - }, - "net": { - "value": 4340.81, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "11466000", - "display": "Cesarean section" - }, - "net": { - "value": 9307.60, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154", - "resource": { - "resourceType": "Encounter", - "id": "014a64da-89fc-4a0b-b2c1-7f6b589da154", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "169762003", - "display": "Postnatal visit" - } - ], - "text": "Postnatal visit" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-06-28T12:47:52-07:00", - "end": "2013-06-28T13:32:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:644d62c7-30a9-4eb7-84b5-0763637ebb17", - "resource": { - "resourceType": "Procedure", - "id": "644d62c7-30a9-4eb7-84b5-0763637ebb17", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination following birth" - } - ], - "text": "Physical examination following birth" - }, - "performedPeriod": { - "start": "2013-06-28T12:47:52-07:00", - "end": "2013-06-28T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:15d7b689-dedc-4132-b6ae-465f68b7bf82", - "resource": { - "resourceType": "Procedure", - "id": "15d7b689-dedc-4132-b6ae-465f68b7bf82", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "171207006", - "display": "Depression screening" - } - ], - "text": "Depression screening" - }, - "performedPeriod": { - "start": "2013-06-28T12:47:52-07:00", - "end": "2013-06-28T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:014a64da-89fc-4a0b-b2c1-7f6b589da154" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c797851e-eaae-4b8c-8ac6-95b2cf6563da", - "resource": { - "resourceType": "Claim", - "id": "c797851e-eaae-4b8c-8ac6-95b2cf6563da", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "169762003", - "display": "Postnatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination following birth" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "171207006", - "display": "Depression screening" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427", - "resource": { - "resourceType": "Encounter", - "id": "1a862e3a-bd75-4182-bb4f-924462d76427", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-07-03T12:47:52-07:00", - "end": "2013-07-03T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a7ccf351-7ba6-4dee-a64c-8a81a715054a", - "resource": { - "resourceType": "Condition", - "id": "a7ccf351-7ba6-4dee-a64c-8a81a715054a", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427" - }, - "dateRecorded": "2013-07-08", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2013-07-08T12:47:52-07:00", - "abatementDateTime": "2013-07-08T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:e7f54cbf-af1a-4798-bc8c-47a1abce8645", - "resource": { - "resourceType": "MedicationOrder", - "id": "e7f54cbf-af1a-4798-bc8c-47a1abce8645", - "dateWritten": "2013-07-03T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:1a862e3a-bd75-4182-bb4f-924462d76427" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "748962", - "display": "Camila 28 Day Pack" - } - ], - "text": "Camila 28 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:aaf2302b-2687-453e-bf5e-cf002712b3f8", - "resource": { - "resourceType": "Claim", - "id": "aaf2302b-2687-453e-bf5e-cf002712b3f8", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:e7f54cbf-af1a-4798-bc8c-47a1abce8645" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:69b20101-431d-45e6-b688-f7b65c7be1c3", - "resource": { - "resourceType": "Claim", - "id": "69b20101-431d-45e6-b688-f7b65c7be1c3", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9", - "resource": { - "resourceType": "Encounter", - "id": "cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2013-07-08T12:47:52-07:00", - "end": "2013-07-08T14:03:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:dfbe2e5f-246e-49c5-a9a8-dd5941574228", - "resource": { - "resourceType": "Observation", - "id": "dfbe2e5f-246e-49c5-a9a8-dd5941574228", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 11.026, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:26f50d1b-3f8a-4ca2-a44c-13528fd7761d", - "resource": { - "resourceType": "Observation", - "id": "26f50d1b-3f8a-4ca2-a44c-13528fd7761d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20570-8", - "display": "Hematocrit [Volume Fraction] of Blood" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 35.399, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb2b41d8-2b99-41e0-b9a9-17b39f7056b0", - "resource": { - "resourceType": "Observation", - "id": "fb2b41d8-2b99-41e0-b9a9-17b39f7056b0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "WBC Auto (Bld) [#/Vol]" - } - ], - "text": "WBC Auto (Bld) [#/Vol]" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 7.8841, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8afdd8e-e6c6-414f-b6bc-4c732b4e55c8", - "resource": { - "resourceType": "Observation", - "id": "d8afdd8e-e6c6-414f-b6bc-4c732b4e55c8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "RBC Auto (Bld) [#/Vol]" - } - ], - "text": "RBC Auto (Bld) [#/Vol]" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 5.0896, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c867668-9947-4bbd-8745-03cb2ca64f30", - "resource": { - "resourceType": "Observation", - "id": "3c867668-9947-4bbd-8745-03cb2ca64f30", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 88.709, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b67eca8a-856e-4d9f-8c93-b4d1c4d913c2", - "resource": { - "resourceType": "Observation", - "id": "b67eca8a-856e-4d9f-8c93-b4d1c4d913c2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 29.414, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7fe15fab-c5ef-4ca5-a4a7-1a7735be85e1", - "resource": { - "resourceType": "Observation", - "id": "7fe15fab-c5ef-4ca5-a4a7-1a7735be85e1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 33.768, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5540aed0-861a-41c3-8e96-09c220ee0b27", - "resource": { - "resourceType": "Observation", - "id": "5540aed0-861a-41c3-8e96-09c220ee0b27", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "RDW - Erythrocyte distribution width Auto (RBC) [Entitic vol]" - } - ], - "text": "RDW - Erythrocyte distribution width Auto (RBC) [Entitic vol]" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 42.075, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:784af967-65b2-4d1b-b6f0-222df46bf033", - "resource": { - "resourceType": "Observation", - "id": "784af967-65b2-4d1b-b6f0-222df46bf033", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 305.64, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:19c1e4da-f18c-4421-8a7f-6aac97530f10", - "resource": { - "resourceType": "Observation", - "id": "19c1e4da-f18c-4421-8a7f-6aac97530f10", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 517.81, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:578d9ea5-43fb-4f5a-aea4-cdd190e094b6", - "resource": { - "resourceType": "Observation", - "id": "578d9ea5-43fb-4f5a-aea4-cdd190e094b6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "valueQuantity": { - "value": 11.873, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2a901aa2-8934-4452-99e2-2554933793e2", - "resource": { - "resourceType": "Procedure", - "id": "2a901aa2-8934-4452-99e2-2554933793e2", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "14768001", - "display": "Peripheral blood smear interpretation" - } - ], - "text": "Peripheral blood smear interpretation" - }, - "performedPeriod": { - "start": "2013-07-08T12:47:52-07:00", - "end": "2013-07-08T13:17:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:844b8b17-c7a4-4519-93b8-da01266836bf", - "resource": { - "resourceType": "Procedure", - "id": "844b8b17-c7a4-4519-93b8-da01266836bf", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "415300000", - "display": "Review of systems (procedure)" - } - ], - "text": "Review of systems (procedure)" - }, - "performedPeriod": { - "start": "2013-07-08T12:47:52-07:00", - "end": "2013-07-08T12:57:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7b812be3-2ec0-4f6d-ad4a-488011b5d315", - "resource": { - "resourceType": "Procedure", - "id": "7b812be3-2ec0-4f6d-ad4a-488011b5d315", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2013-07-08T12:47:52-07:00", - "end": "2013-07-08T12:55:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a4d661fe-489b-4acd-9a59-f530a23df565", - "resource": { - "resourceType": "Procedure", - "id": "a4d661fe-489b-4acd-9a59-f530a23df565", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162676008", - "display": "Brief general examination (procedure)" - } - ], - "text": "Brief general examination (procedure)" - }, - "performedPeriod": { - "start": "2013-07-08T12:47:52-07:00", - "end": "2013-07-08T13:00:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a9cb7d8a-de37-4f1d-88f4-8723c218f7e0", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a9cb7d8a-de37-4f1d-88f4-8723c218f7e0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:cb1a0008-d6d6-4554-b4ce-0a3a42e9ccc9" - }, - "effectiveDateTime": "2013-07-08T12:47:52-07:00", - "issued": "2013-07-08T12:47:52.730-07:00", - "performer": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "result": [ - { - "reference": "urn:uuid:578d9ea5-43fb-4f5a-aea4-cdd190e094b6", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0c435b76-697a-4514-8d0a-0a5a42b4e0e7", - "resource": { - "resourceType": "Claim", - "id": "0c435b76-697a-4514-8d0a-0a5a42b4e0e7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "14768001", - "display": "Peripheral blood smear interpretation" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "415300000", - "display": "Review of systems (procedure)" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 469.21, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162676008", - "display": "Brief general examination (procedure)" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6b7383a6-0ded-408e-8701-d1e72f17697a", - "resource": { - "resourceType": "Encounter", - "id": "6b7383a6-0ded-408e-8701-d1e72f17697a", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2014-06-28T12:47:52-07:00", - "end": "2014-06-28T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d", - "resource": { - "resourceType": "MedicationOrder", - "id": "0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d", - "dateWritten": "2014-06-28T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:6b7383a6-0ded-408e-8701-d1e72f17697a" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "757594", - "display": "Jolivette 28 Day Pack" - } - ], - "text": "Jolivette 28 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:0cb4cac2-eb7d-4da1-b120-22df6a12f592", - "resource": { - "resourceType": "Claim", - "id": "0cb4cac2-eb7d-4da1-b120-22df6a12f592", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:0b1d16ba-672f-4d7f-8a1c-576fbc55ff7d" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8a26487b-9cfd-425b-b3aa-260e7c21862a", - "resource": { - "resourceType": "Claim", - "id": "8a26487b-9cfd-425b-b3aa-260e7c21862a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e19a6916-47ef-433a-b37a-1c4ff1f2729a", - "resource": { - "resourceType": "Encounter", - "id": "e19a6916-47ef-433a-b37a-1c4ff1f2729a", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2015-06-23T12:47:52-07:00", - "end": "2015-06-23T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c9f6b596-614a-456b-8133-de75146f8710", - "resource": { - "resourceType": "MedicationOrder", - "id": "c9f6b596-614a-456b-8133-de75146f8710", - "dateWritten": "2015-06-23T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:e19a6916-47ef-433a-b37a-1c4ff1f2729a" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1367439", - "display": "NuvaRing 0.12/0.015 MG per 24HR 21 Day Vaginal Ring" - } - ], - "text": "NuvaRing 0.12/0.015 MG per 24HR 21 Day Vaginal Ring" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:d24a3c2a-24ad-48da-b720-c04d2c58ece4", - "resource": { - "resourceType": "Claim", - "id": "d24a3c2a-24ad-48da-b720-c04d2c58ece4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:c9f6b596-614a-456b-8133-de75146f8710" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:214e96bb-b248-4a18-a0d0-4c71bf3fc1e5", - "resource": { - "resourceType": "Claim", - "id": "214e96bb-b248-4a18-a0d0-4c71bf3fc1e5", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642", - "resource": { - "resourceType": "Encounter", - "id": "e396cffd-247d-4fdd-858a-5fa85574c642", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" - } - } - ], - "period": { - "start": "2015-07-24T12:47:52-07:00", - "end": "2015-07-24T13:17:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f574d297-e04a-482a-ab71-198be984cdb2", - "resource": { - "resourceType": "Observation", - "id": "f574d297-e04a-482a-ab71-198be984cdb2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 162.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:370f86e4-09cb-4ede-8a32-a40b0c6930b3", - "resource": { - "resourceType": "Observation", - "id": "370f86e4-09cb-4ede-8a32-a40b0c6930b3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30f7d65b-95ae-42a2-ab36-972bef42df71", - "resource": { - "resourceType": "Observation", - "id": "30f7d65b-95ae-42a2-ab36-972bef42df71", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 79.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5066aa49-ccfb-425a-8c1d-6a5a04eb0b16", - "resource": { - "resourceType": "Observation", - "id": "5066aa49-ccfb-425a-8c1d-6a5a04eb0b16", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 30.030, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb62634b-be80-42ef-951d-01306a1bd9e3", - "resource": { - "resourceType": "Observation", - "id": "cb62634b-be80-42ef-951d-01306a1bd9e3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4cd15784-77b5-4dc3-8f3a-8562c27329d5", - "resource": { - "resourceType": "Observation", - "id": "4cd15784-77b5-4dc3-8f3a-8562c27329d5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 88, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31abcec9-89cd-441d-9c6e-755149c94baa", - "resource": { - "resourceType": "Observation", - "id": "31abcec9-89cd-441d-9c6e-755149c94baa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d5ed4719-611c-4ff2-b79b-0675dc590183", - "resource": { - "resourceType": "Observation", - "id": "d5ed4719-611c-4ff2-b79b-0675dc590183", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 3.7063, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbec67f1-5e49-4a27-bc2f-7c38f9bee7df", - "resource": { - "resourceType": "Observation", - "id": "bbec67f1-5e49-4a27-bc2f-7c38f9bee7df", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 4.5130, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d10dabf-8491-4775-a4e8-9d05945a9d76", - "resource": { - "resourceType": "Observation", - "id": "3d10dabf-8491-4775-a4e8-9d05945a9d76", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 14.989, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b7152cc-b9d9-4eaf-98d0-81682aac404b", - "resource": { - "resourceType": "Observation", - "id": "2b7152cc-b9d9-4eaf-98d0-81682aac404b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 41.010, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65c103ab-e6d1-442e-a710-721df8495e1d", - "resource": { - "resourceType": "Observation", - "id": "65c103ab-e6d1-442e-a710-721df8495e1d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 89.560, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82f24393-cea1-4342-9fb3-c4845136a5e7", - "resource": { - "resourceType": "Observation", - "id": "82f24393-cea1-4342-9fb3-c4845136a5e7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 27.456, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0368e7cf-3b1f-413d-950a-89a2c370cc2d", - "resource": { - "resourceType": "Observation", - "id": "0368e7cf-3b1f-413d-950a-89a2c370cc2d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 35.103, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33e621d9-4bc2-4f69-9987-c93786e01970", - "resource": { - "resourceType": "Observation", - "id": "33e621d9-4bc2-4f69-9987-c93786e01970", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 40.945, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7a07912-88a6-4ce6-89b2-d467ac17fd2f", - "resource": { - "resourceType": "Observation", - "id": "b7a07912-88a6-4ce6-89b2-d467ac17fd2f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 167.85, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2cff4c7-a35b-4be1-9cd4-e06840213f3d", - "resource": { - "resourceType": "Observation", - "id": "b2cff4c7-a35b-4be1-9cd4-e06840213f3d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 487.14, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d000ca8-1d46-4c7e-a5a1-4dede883d961", - "resource": { - "resourceType": "Observation", - "id": "1d000ca8-1d46-4c7e-a5a1-4dede883d961", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueQuantity": { - "value": 11.167, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:79653c9e-a2d2-42c1-b765-7da6f45ed748", - "resource": { - "resourceType": "Observation", - "id": "79653c9e-a2d2-42c1-b765-7da6f45ed748", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8d07f36d-450c-4f9a-9809-a98f792bfa5f", - "resource": { - "resourceType": "Procedure", - "id": "8d07f36d-450c-4f9a-9809-a98f792bfa5f", - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2015-07-24T12:47:52-07:00", - "end": "2015-07-24T13:02:52-07:00" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ba21d7c7-2d01-43ed-bf69-c7ff9b6a5a97", - "resource": { - "resourceType": "Immunization", - "id": "ba21d7c7-2d01-43ed-bf69-c7ff9b6a5a97", - "status": "completed", - "date": "2015-07-24T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:bb99b956-fc6c-4023-9745-3c84b96c41cd", - "resource": { - "resourceType": "Immunization", - "id": "bb99b956-fc6c-4023-9745-3c84b96c41cd", - "status": "completed", - "date": "2015-07-24T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - } - ], - "text": "Hep A, adult" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a70d175a-b888-41e1-b0c6-03c6c16097b9", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a70d175a-b888-41e1-b0c6-03c6c16097b9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:e396cffd-247d-4fdd-858a-5fa85574c642" - }, - "effectiveDateTime": "2015-07-24T12:47:52-07:00", - "issued": "2015-07-24T12:47:52.730-07:00", - "performer": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "result": [ - { - "reference": "urn:uuid:1d000ca8-1d46-4c7e-a5a1-4dede883d961", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f1a78a45-429a-41cd-8416-106d9948e2ba", - "resource": { - "resourceType": "Claim", - "id": "f1a78a45-429a-41cd-8416-106d9948e2ba", - "type": "institutional", - "organization": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 454.36, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e", - "resource": { - "resourceType": "Encounter", - "id": "a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2016-06-17T12:47:52-07:00", - "end": "2016-06-17T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:262360cb-9fc9-496c-aa0b-b81d8ae62060", - "resource": { - "resourceType": "MedicationOrder", - "id": "262360cb-9fc9-496c-aa0b-b81d8ae62060", - "dateWritten": "2016-06-17T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:a2b27123-8fb3-4b68-bdc2-ef4fd71ec40e" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "757594", - "display": "Jolivette 28 Day Pack" - } - ], - "text": "Jolivette 28 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:b3639899-7c53-423d-86ed-352c8f5af43c", - "resource": { - "resourceType": "Claim", - "id": "b3639899-7c53-423d-86ed-352c8f5af43c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:262360cb-9fc9-496c-aa0b-b81d8ae62060" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2430fd45-8ca8-4d54-b6fb-76b5448d0e2c", - "resource": { - "resourceType": "Claim", - "id": "2430fd45-8ca8-4d54-b6fb-76b5448d0e2c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048", - "resource": { - "resourceType": "Encounter", - "id": "9e720654-ab2d-4c64-a0c5-92912f21d048", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2016-08-02T12:47:52-07:00", - "end": "2016-08-02T13:02:52-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5798b5d3-df47-4eb8-9ad5-aea32bfce5b0", - "resource": { - "resourceType": "Condition", - "id": "5798b5d3-df47-4eb8-9ad5-aea32bfce5b0", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048" - }, - "dateRecorded": "2016-08-02", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2016-08-02T12:47:52-07:00", - "abatementDateTime": "2016-08-11T12:47:52-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:496feb8f-6322-43c2-90e0-adc05a760ccc", - "resource": { - "resourceType": "Observation", - "id": "496feb8f-6322-43c2-90e0-adc05a760ccc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:9e720654-ab2d-4c64-a0c5-92912f21d048" - }, - "effectiveDateTime": "2016-08-02T12:47:52-07:00", - "issued": "2016-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 37.502, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92a34b1c-fd71-4f23-aaa5-304f49968548", - "resource": { - "resourceType": "Claim", - "id": "92a34b1c-fd71-4f23-aaa5-304f49968548", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:94372af5-4177-4757-991e-eeab8666cb4d", - "resource": { - "resourceType": "Encounter", - "id": "94372af5-4177-4757-991e-eeab8666cb4d", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2017-06-12T12:47:52-07:00", - "end": "2017-06-12T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d68f95cd-cdc6-4f3a-a9df-4c64355f0d62", - "resource": { - "resourceType": "MedicationOrder", - "id": "d68f95cd-cdc6-4f3a-a9df-4c64355f0d62", - "dateWritten": "2017-06-12T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:94372af5-4177-4757-991e-eeab8666cb4d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "751905", - "display": "Trinessa 28 Day Pack" - } - ], - "text": "Trinessa 28 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:74728217-e9d5-476e-9cd4-50dbcdc23b29", - "resource": { - "resourceType": "Claim", - "id": "74728217-e9d5-476e-9cd4-50dbcdc23b29", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:d68f95cd-cdc6-4f3a-a9df-4c64355f0d62" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:96d694e8-1878-4096-a912-5d5262c9919a", - "resource": { - "resourceType": "Claim", - "id": "96d694e8-1878-4096-a912-5d5262c9919a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762", - "resource": { - "resourceType": "Encounter", - "id": "ed971cc5-27c9-41d4-a5ef-f1d22a097762", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" - } - } - ], - "period": { - "start": "2017-07-28T12:47:52-07:00", - "end": "2017-07-28T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:513096da-0208-41fb-a191-92d61ec98d9d", - "resource": { - "resourceType": "Observation", - "id": "513096da-0208-41fb-a191-92d61ec98d9d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 162.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:24b3d8a6-fd2d-4c9f-9ee4-3b0faf9d94f8", - "resource": { - "resourceType": "Observation", - "id": "24b3d8a6-fd2d-4c9f-9ee4-3b0faf9d94f8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36d4bbd6-a005-4c3d-bc90-90b5bef3859b", - "resource": { - "resourceType": "Observation", - "id": "36d4bbd6-a005-4c3d-bc90-90b5bef3859b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 72.700, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3c7de8e-7577-483c-a5e3-b0a4547f646a", - "resource": { - "resourceType": "Observation", - "id": "b3c7de8e-7577-483c-a5e3-b0a4547f646a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 27.430, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:066f2d74-e6c8-423b-ba18-daf2bc55962f", - "resource": { - "resourceType": "Observation", - "id": "066f2d74-e6c8-423b-ba18-daf2bc55962f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68245bf5-1481-4a79-92b1-f37c4854ff06", - "resource": { - "resourceType": "Observation", - "id": "68245bf5-1481-4a79-92b1-f37c4854ff06", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 98, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59458042-1e08-4baa-b57b-0da4f0999cd9", - "resource": { - "resourceType": "Observation", - "id": "59458042-1e08-4baa-b57b-0da4f0999cd9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cb4838e-a1ce-4d2d-b7b7-a3e1510ac22b", - "resource": { - "resourceType": "Observation", - "id": "0cb4838e-a1ce-4d2d-b7b7-a3e1510ac22b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 187.95, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8bc571a6-8900-43f4-9de6-16e9e344c153", - "resource": { - "resourceType": "Observation", - "id": "8bc571a6-8900-43f4-9de6-16e9e344c153", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 148.96, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f10ce4b4-57db-4aa8-b9a3-1f475d97a463", - "resource": { - "resourceType": "Observation", - "id": "f10ce4b4-57db-4aa8-b9a3-1f475d97a463", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 89.710, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c74ce653-d7ec-4d3c-bead-2cac242570d1", - "resource": { - "resourceType": "Observation", - "id": "c74ce653-d7ec-4d3c-bead-2cac242570d1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueQuantity": { - "value": 68.450, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d983d75-de81-4b9c-9aa4-a4c02c36afdd", - "resource": { - "resourceType": "Observation", - "id": "9d983d75-de81-4b9c-9aa4-a4c02c36afdd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ccbfc84-232a-4033-b136-2c5ff11b6a7f", - "resource": { - "resourceType": "Immunization", - "id": "9ccbfc84-232a-4033-b136-2c5ff11b6a7f", - "status": "completed", - "date": "2017-07-28T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ac725ef6-5617-4b54-872c-7b2b0c6e8827", - "resource": { - "resourceType": "Immunization", - "id": "ac725ef6-5617-4b54-872c-7b2b0c6e8827", - "status": "completed", - "date": "2017-07-28T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f743fe11-6db2-4b06-8c84-e26401cbfe5c", - "resource": { - "resourceType": "Immunization", - "id": "f743fe11-6db2-4b06-8c84-e26401cbfe5c", - "status": "completed", - "date": "2017-07-28T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - } - ], - "text": "Hep A, adult" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:79bfa9f5-8a60-4296-8969-1ed167e5f550", - "resource": { - "resourceType": "DiagnosticReport", - "id": "79bfa9f5-8a60-4296-8969-1ed167e5f550", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ed971cc5-27c9-41d4-a5ef-f1d22a097762" - }, - "effectiveDateTime": "2017-07-28T12:47:52-07:00", - "issued": "2017-07-28T12:47:52.730-07:00", - "performer": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "result": [ - { - "reference": "urn:uuid:c74ce653-d7ec-4d3c-bead-2cac242570d1", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:c2823519-9690-4b77-9f15-cbecf0fd2a6d", - "resource": { - "resourceType": "Claim", - "id": "c2823519-9690-4b77-9f15-cbecf0fd2a6d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8b4c29d4-d667-450c-96d8-bb3b04255396", - "resource": { - "resourceType": "Encounter", - "id": "8b4c29d4-d667-450c-96d8-bb3b04255396", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2018-06-07T12:47:52-07:00", - "end": "2018-06-07T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bc75a463-d467-4698-afa1-d22b5aeaff60", - "resource": { - "resourceType": "MedicationOrder", - "id": "bc75a463-d467-4698-afa1-d22b5aeaff60", - "dateWritten": "2018-06-07T12:47:52-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:8b4c29d4-d667-450c-96d8-bb3b04255396" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "978950", - "display": "Natazia 28 Day Pack" - } - ], - "text": "Natazia 28 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:7c8893ed-2611-41eb-bf80-6fe6b7b992c8", - "resource": { - "resourceType": "Claim", - "id": "7c8893ed-2611-41eb-bf80-6fe6b7b992c8", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:bc75a463-d467-4698-afa1-d22b5aeaff60" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c625cf6f-ad18-49ae-ac3b-ef257f0f3f1c", - "resource": { - "resourceType": "Claim", - "id": "c625cf6f-ad18-49ae-ac3b-ef257f0f3f1c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a44a3669-8d9e-4e72-8733-f85346a8688d", - "resource": { - "resourceType": "Encounter", - "id": "a44a3669-8d9e-4e72-8733-f85346a8688d", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - } - } - ], - "period": { - "start": "2019-06-02T12:47:52-07:00", - "end": "2019-06-02T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:044faa36-ce9d-4c99-9de1-f629f43d4d01", - "resource": { - "resourceType": "MedicationOrder", - "id": "044faa36-ce9d-4c99-9de1-f629f43d4d01", - "dateWritten": "2019-06-02T12:47:52-07:00", - "status": "active", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "prescriber": { - "reference": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e" - }, - "encounter": { - "reference": "urn:uuid:a44a3669-8d9e-4e72-8733-f85346a8688d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "749762", - "display": "Seasonique 91 Day Pack" - } - ], - "text": "Seasonique 91 Day Pack" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:3283f083-75c4-45f6-858a-718e373d599b", - "resource": { - "resourceType": "Claim", - "id": "3283f083-75c4-45f6-858a-718e373d599b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:044faa36-ce9d-4c99-9de1-f629f43d4d01" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:608ba58c-06e0-4426-86ce-08f4ec7815fa", - "resource": { - "resourceType": "Claim", - "id": "608ba58c-06e0-4426-86ce-08f4ec7815fa", - "type": "institutional", - "organization": { - "reference": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c", - "resource": { - "resourceType": "Encounter", - "id": "ac8f9312-f86a-47b6-9af1-2359e349d65c", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc" - } - } - ], - "period": { - "start": "2019-08-02T12:47:52-07:00", - "end": "2019-08-02T13:02:52-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5f02d23a-f841-474b-b1a0-5e4b01e73ebd", - "resource": { - "resourceType": "Observation", - "id": "5f02d23a-f841-474b-b1a0-5e4b01e73ebd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 162.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aff5cc5f-74d9-4347-ad17-7025ed997578", - "resource": { - "resourceType": "Observation", - "id": "aff5cc5f-74d9-4347-ad17-7025ed997578", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11c3c9c0-d15b-46b5-8f35-ff534134ac1a", - "resource": { - "resourceType": "Observation", - "id": "11c3c9c0-d15b-46b5-8f35-ff534134ac1a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 76.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c94e846-884d-45f5-bbb7-21fa3c4ea842", - "resource": { - "resourceType": "Observation", - "id": "9c94e846-884d-45f5-bbb7-21fa3c4ea842", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 28.730, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b732f3de-8ad0-4f30-9080-a42b1a80948b", - "resource": { - "resourceType": "Observation", - "id": "b732f3de-8ad0-4f30-9080-a42b1a80948b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 132, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f9e36a87-9b2b-40c1-b818-371411feb397", - "resource": { - "resourceType": "Observation", - "id": "f9e36a87-9b2b-40c1-b818-371411feb397", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d6a70dc-c941-4414-9919-7d26319a9939", - "resource": { - "resourceType": "Observation", - "id": "0d6a70dc-c941-4414-9919-7d26319a9939", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:81a12a37-720e-4bee-8b81-ef7b7403c90a", - "resource": { - "resourceType": "Observation", - "id": "81a12a37-720e-4bee-8b81-ef7b7403c90a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - }, - "effectiveDateTime": "2019-08-02T12:47:52-07:00", - "issued": "2019-08-02T12:47:52.730-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82e2f210-8197-40b1-bef2-907f7d800975", - "resource": { - "resourceType": "Immunization", - "id": "82e2f210-8197-40b1-bef2-907f7d800975", - "status": "completed", - "date": "2019-08-02T12:47:52-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ac8f9312-f86a-47b6-9af1-2359e349d65c" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:653af184-26f4-47a6-adc2-1648928d6be4", - "resource": { - "resourceType": "Claim", - "id": "653af184-26f4-47a6-adc2-1648928d6be4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:1fd2683f-2a56-47c7-b674-98f2cc8319e7" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json deleted file mode 100644 index ac94ea8d8adc..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Ernesto186_Dietrich576_4ecb4cbb-6df7-41e0-8e89-6e7a142721a5.json +++ /dev/null @@ -1,17399 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5", - "resource": { - "resourceType": "Patient", - "id": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 5597799561560712484 Population seed: 1586309754086
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Race", - "code": "2106-3", - "display": "White" - } - ], - "text": "White" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Ethnicity", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - ], - "text": "Not Hispanic or Latino" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Cindie288 Beer512" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "M" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Easton", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.20976015150670563 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 8.790239848493295 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR" - } - ] - }, - "system": "http://hospital.smarthealthit.org", - "value": "4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB" - } - ] - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-15-6629" - } - ], - "name": [ - { - "use": "official", - "family": [ - "Dietrich576" - ], - "given": [ - "Ernesto186" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-410-4513", - "use": "home" - } - ], - "gender": "male", - "birthDate": "2010-11-25", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.328966094980274 - }, - { - "url": "longitude", - "valueDecimal": -71.06791912415248 - } - ] - } - ], - "line": [ - "784 Johns Tunnel Apt 71" - ], - "city": "Boston", - "state": "Massachusetts", - "postalCode": "02110", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S" - } - ] - }, - "multipleBirthInteger": 2, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86", - "resource": { - "resourceType": "Organization", - "id": "70d8b93b-cc76-3c8b-8929-5aea213ecc86", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "BROOKLINE DERMATOLOGY ASSOCIATES, PC", - "telecom": [ - { - "system": "phone", - "value": "617-608-1575" - } - ], - "address": [ - { - "line": [ - "1208 B VFW PKWY" - ], - "city": "WEST ROXBURY", - "state": "MA", - "postalCode": "02132-4350", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e", - "resource": { - "resourceType": "Practitioner", - "id": "2d307854-d23a-337c-8215-e37f84fdc26e", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "66350" - } - ], - "active": true, - "name": { - "family": [ - "Eichmann909" - ], - "given": [ - "Marylou497" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "1208 B VFW PKWY" - ], - "city": "WEST ROXBURY", - "state": "MA", - "postalCode": "02132-4350", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58", - "resource": { - "resourceType": "Encounter", - "id": "2c1324ea-612c-4d84-849e-009205ea3b58", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2010-11-25T11:17:06-08:00", - "end": "2010-11-25T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8ebf2c45-def4-4c23-8e3f-c0e4ac79c959", - "resource": { - "resourceType": "Observation", - "id": "8ebf2c45-def4-4c23-8e3f-c0e4ac79c959", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 50.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fceaa3b5-ca1a-4e19-9e30-dba58f9e71e5", - "resource": { - "resourceType": "Observation", - "id": "fceaa3b5-ca1a-4e19-9e30-dba58f9e71e5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f9db9937-9f87-421a-a6b5-0cb08e75ebcd", - "resource": { - "resourceType": "Observation", - "id": "f9db9937-9f87-421a-a6b5-0cb08e75ebcd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 3.8000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:027003ed-8bd3-420f-9fe9-f0d9b521d2f2", - "resource": { - "resourceType": "Observation", - "id": "027003ed-8bd3-420f-9fe9-f0d9b521d2f2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 78.413, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d20b1c0-8d7e-42be-8f9d-0a258cb8fa39", - "resource": { - "resourceType": "Observation", - "id": "5d20b1c0-8d7e-42be-8f9d-0a258cb8fa39", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 33.860, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:16393f99-2b04-4a15-8d70-404521828062", - "resource": { - "resourceType": "Observation", - "id": "16393f99-2b04-4a15-8d70-404521828062", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 116, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33447bb9-8c10-430f-836e-fee7cf00820b", - "resource": { - "resourceType": "Observation", - "id": "33447bb9-8c10-430f-836e-fee7cf00820b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 95, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:20cf6011-db84-412c-b831-391592133d0d", - "resource": { - "resourceType": "Observation", - "id": "20cf6011-db84-412c-b831-391592133d0d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50934ad5-66ec-463a-a1c5-0bb9eb23a310", - "resource": { - "resourceType": "Observation", - "id": "50934ad5-66ec-463a-a1c5-0bb9eb23a310", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 6.4777, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4ab365d2-296f-4ac8-9e5a-ac816659a12e", - "resource": { - "resourceType": "Observation", - "id": "4ab365d2-296f-4ac8-9e5a-ac816659a12e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 4.0205, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d5efd4a1-76aa-4a08-ba76-99de85d0b502", - "resource": { - "resourceType": "Observation", - "id": "d5efd4a1-76aa-4a08-ba76-99de85d0b502", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 13.276, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9e09db52-230b-4bfe-b0aa-ad5b073d2711", - "resource": { - "resourceType": "Observation", - "id": "9e09db52-230b-4bfe-b0aa-ad5b073d2711", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 37.735, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:63c1447e-6a0a-4954-a858-c703124b87a5", - "resource": { - "resourceType": "Observation", - "id": "63c1447e-6a0a-4954-a858-c703124b87a5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 89.968, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99c12063-3565-48b2-8558-99f45b9a7973", - "resource": { - "resourceType": "Observation", - "id": "99c12063-3565-48b2-8558-99f45b9a7973", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 32.199, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c8c0590-e592-427f-adb6-95440513b9ea", - "resource": { - "resourceType": "Observation", - "id": "4c8c0590-e592-427f-adb6-95440513b9ea", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 34.686, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0ac6566-41b0-401f-958d-abfe390ca4b0", - "resource": { - "resourceType": "Observation", - "id": "c0ac6566-41b0-401f-958d-abfe390ca4b0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 41.069, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0a165cc6-dad1-4f9e-b439-278351ab9138", - "resource": { - "resourceType": "Observation", - "id": "0a165cc6-dad1-4f9e-b439-278351ab9138", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 199.09, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7ac73c3-3dd0-4432-998f-ff713ac3e537", - "resource": { - "resourceType": "Observation", - "id": "a7ac73c3-3dd0-4432-998f-ff713ac3e537", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 388.77, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:121bb5d6-d2ae-428e-8209-e5eb73f2152f", - "resource": { - "resourceType": "Observation", - "id": "121bb5d6-d2ae-428e-8209-e5eb73f2152f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueQuantity": { - "value": 9.6619, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8921de55-87a0-4e6e-bce4-1dee4371a66f", - "resource": { - "resourceType": "Observation", - "id": "8921de55-87a0-4e6e-bce4-1dee4371a66f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65cc1f49-4886-4a84-bd17-1401b24835ce", - "resource": { - "resourceType": "Procedure", - "id": "65cc1f49-4886-4a84-bd17-1401b24835ce", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2010-11-25T11:17:06-08:00", - "end": "2010-11-25T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c6219428-432d-4ccf-88a1-239d079e786a", - "resource": { - "resourceType": "Immunization", - "id": "c6219428-432d-4ccf-88a1-239d079e786a", - "status": "completed", - "date": "2010-11-25T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:33f4287c-c55e-472f-8a6d-2d924be47b39", - "resource": { - "resourceType": "DiagnosticReport", - "id": "33f4287c-c55e-472f-8a6d-2d924be47b39", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2c1324ea-612c-4d84-849e-009205ea3b58" - }, - "effectiveDateTime": "2010-11-25T11:17:06-08:00", - "issued": "2010-11-25T11:17:06.181-08:00", - "performer": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "result": [ - { - "reference": "urn:uuid:121bb5d6-d2ae-428e-8209-e5eb73f2152f", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:758d41bd-402c-4fb7-b8a0-d0a2e3899b15", - "resource": { - "resourceType": "Claim", - "id": "758d41bd-402c-4fb7-b8a0-d0a2e3899b15", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 764.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c", - "resource": { - "resourceType": "Encounter", - "id": "9f97ee46-dcd4-4c81-a02b-fee0eff9f13c", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2010-12-30T11:17:06-08:00", - "end": "2010-12-30T11:32:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:17830cab-4bf5-48e0-8daa-baceb286132a", - "resource": { - "resourceType": "Observation", - "id": "17830cab-4bf5-48e0-8daa-baceb286132a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 54.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7aeff5d2-982c-4053-adc3-cff39eb60f20", - "resource": { - "resourceType": "Observation", - "id": "7aeff5d2-982c-4053-adc3-cff39eb60f20", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bab19a92-d92c-4ecc-b56b-bc7d8f7bc65c", - "resource": { - "resourceType": "Observation", - "id": "bab19a92-d92c-4ecc-b56b-bc7d8f7bc65c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 4.7000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fe2478d5-2312-4058-881a-5c3c8fca88b3", - "resource": { - "resourceType": "Observation", - "id": "fe2478d5-2312-4058-881a-5c3c8fca88b3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 69.568, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5665c9e0-2d96-461c-98bb-a93e7a5b4e66", - "resource": { - "resourceType": "Observation", - "id": "5665c9e0-2d96-461c-98bb-a93e7a5b4e66", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 37.580, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:00091da9-8395-40b3-8e0b-53f6bd0c2ec7", - "resource": { - "resourceType": "Observation", - "id": "00091da9-8395-40b3-8e0b-53f6bd0c2ec7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 133, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35c6de82-2533-4b1b-9c3c-64062b2263d7", - "resource": { - "resourceType": "Observation", - "id": "35c6de82-2533-4b1b-9c3c-64062b2263d7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 67, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:708c78c2-9ab4-4c76-b2e2-b0cd4e8b0a49", - "resource": { - "resourceType": "Observation", - "id": "708c78c2-9ab4-4c76-b2e2-b0cd4e8b0a49", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cea9b66e-93f5-4a92-8f75-6196465ac1fa", - "resource": { - "resourceType": "Observation", - "id": "cea9b66e-93f5-4a92-8f75-6196465ac1fa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - }, - "effectiveDateTime": "2010-12-30T11:17:06-08:00", - "issued": "2010-12-30T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9582a8d2-5838-4ab2-bf58-ea4ca0326490", - "resource": { - "resourceType": "Immunization", - "id": "9582a8d2-5838-4ab2-bf58-ea4ca0326490", - "status": "completed", - "date": "2010-12-30T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:9f97ee46-dcd4-4c81-a02b-fee0eff9f13c" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:9ea34bc3-2591-4771-acc8-9458194f5471", - "resource": { - "resourceType": "Claim", - "id": "9ea34bc3-2591-4771-acc8-9458194f5471", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4", - "resource": { - "resourceType": "Encounter", - "id": "98699fd9-2ee9-4308-8b5b-09d9d78bd8d4", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2011-03-03T11:17:06-08:00", - "end": "2011-03-03T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cfa21e58-5812-49d9-9e5b-c84795c54fa9", - "resource": { - "resourceType": "Observation", - "id": "cfa21e58-5812-49d9-9e5b-c84795c54fa9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 59.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3813e00-9ad7-421c-928c-2bfa8b403d50", - "resource": { - "resourceType": "Observation", - "id": "f3813e00-9ad7-421c-928c-2bfa8b403d50", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a98bbb5d-7d71-44de-9f4d-07e06e3735af", - "resource": { - "resourceType": "Observation", - "id": "a98bbb5d-7d71-44de-9f4d-07e06e3735af", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 6.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28f83f6a-0a0d-4591-b488-9fd8ed7c4b92", - "resource": { - "resourceType": "Observation", - "id": "28f83f6a-0a0d-4591-b488-9fd8ed7c4b92", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 63.292, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:63e63183-9097-409a-bfce-b8375d2eead8", - "resource": { - "resourceType": "Observation", - "id": "63e63183-9097-409a-bfce-b8375d2eead8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 40.360, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:523aa424-944a-46b5-b84c-f4d13375d373", - "resource": { - "resourceType": "Observation", - "id": "523aa424-944a-46b5-b84c-f4d13375d373", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 133, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2479917-579c-4450-b2b2-5ec99295e050", - "resource": { - "resourceType": "Observation", - "id": "e2479917-579c-4450-b2b2-5ec99295e050", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 77, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7143f445-203a-4523-a367-23986099d586", - "resource": { - "resourceType": "Observation", - "id": "7143f445-203a-4523-a367-23986099d586", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2f0dffcc-2efe-4020-85d8-200f43a1dbc2", - "resource": { - "resourceType": "Observation", - "id": "2f0dffcc-2efe-4020-85d8-200f43a1dbc2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - }, - "effectiveDateTime": "2011-03-03T11:17:06-08:00", - "issued": "2011-03-03T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c1eaee3b-865f-4415-b9c2-5afe3b0f5e94", - "resource": { - "resourceType": "Procedure", - "id": "c1eaee3b-865f-4415-b9c2-5afe3b0f5e94", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2011-03-03T11:17:06-08:00", - "end": "2011-03-03T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9709808b-2050-4721-b5f2-8b7815911f59", - "resource": { - "resourceType": "Immunization", - "id": "9709808b-2050-4721-b5f2-8b7815911f59", - "status": "completed", - "date": "2011-03-03T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b0e4acac-3741-41dd-8c50-22e7e5bb3f89", - "resource": { - "resourceType": "Immunization", - "id": "b0e4acac-3741-41dd-8c50-22e7e5bb3f89", - "status": "completed", - "date": "2011-03-03T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:dd8562b5-6a35-470f-8276-06e35f0033be", - "resource": { - "resourceType": "Immunization", - "id": "dd8562b5-6a35-470f-8276-06e35f0033be", - "status": "completed", - "date": "2011-03-03T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:da335d7d-824c-43d7-b4e6-9310c8575e03", - "resource": { - "resourceType": "Immunization", - "id": "da335d7d-824c-43d7-b4e6-9310c8575e03", - "status": "completed", - "date": "2011-03-03T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:90a87f3a-ef6e-490c-a8e0-773b61238422", - "resource": { - "resourceType": "Immunization", - "id": "90a87f3a-ef6e-490c-a8e0-773b61238422", - "status": "completed", - "date": "2011-03-03T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:98699fd9-2ee9-4308-8b5b-09d9d78bd8d4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b9e4e4e6-26fd-4dbb-862e-332accb0a87f", - "resource": { - "resourceType": "Claim", - "id": "b9e4e4e6-26fd-4dbb-862e-332accb0a87f", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 603.15, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd", - "resource": { - "resourceType": "Encounter", - "id": "31109686-bf0b-410d-9920-1348ec1098fd", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2011-05-05T12:17:06-07:00", - "end": "2011-05-05T12:32:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:570d6b42-b71c-4499-9245-330fa0c5344e", - "resource": { - "resourceType": "Observation", - "id": "570d6b42-b71c-4499-9245-330fa0c5344e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 63.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a105f262-1c6b-4465-862d-f3d496f2e288", - "resource": { - "resourceType": "Observation", - "id": "a105f262-1c6b-4465-862d-f3d496f2e288", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d45089d-c984-42dc-8ac9-33951ec01bac", - "resource": { - "resourceType": "Observation", - "id": "7d45089d-c984-42dc-8ac9-33951ec01bac", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 7.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6af14676-fdce-4728-8705-4aea01e9abef", - "resource": { - "resourceType": "Observation", - "id": "6af14676-fdce-4728-8705-4aea01e9abef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 71.421, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4da7d003-c99a-4830-95df-ff04bd84f3d1", - "resource": { - "resourceType": "Observation", - "id": "4da7d003-c99a-4830-95df-ff04bd84f3d1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 42.110, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7ad1aae-d601-423b-bdbc-6e9207fbfceb", - "resource": { - "resourceType": "Observation", - "id": "a7ad1aae-d601-423b-bdbc-6e9207fbfceb", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbf20ae1-e0b1-48fa-833c-374255ba2fbe", - "resource": { - "resourceType": "Observation", - "id": "bbf20ae1-e0b1-48fa-833c-374255ba2fbe", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 95, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebaca6b5-0ced-4f46-9cb9-7fc4d310de83", - "resource": { - "resourceType": "Observation", - "id": "ebaca6b5-0ced-4f46-9cb9-7fc4d310de83", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8dbcb168-e08d-403c-972d-aac59eaafcc1", - "resource": { - "resourceType": "Observation", - "id": "8dbcb168-e08d-403c-972d-aac59eaafcc1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - }, - "effectiveDateTime": "2011-05-05T12:17:06-07:00", - "issued": "2011-05-05T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d7f7d12f-8064-4a9e-96cd-7648afab0563", - "resource": { - "resourceType": "Immunization", - "id": "d7f7d12f-8064-4a9e-96cd-7648afab0563", - "status": "completed", - "date": "2011-05-05T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0211cf39-777c-4ff4-8100-06486c9f10c6", - "resource": { - "resourceType": "Immunization", - "id": "0211cf39-777c-4ff4-8100-06486c9f10c6", - "status": "completed", - "date": "2011-05-05T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b794328e-9e13-4c6a-9d28-de8699523a71", - "resource": { - "resourceType": "Immunization", - "id": "b794328e-9e13-4c6a-9d28-de8699523a71", - "status": "completed", - "date": "2011-05-05T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2f2e3ad1-b57b-45d2-8f0b-cab25ca48fb7", - "resource": { - "resourceType": "Immunization", - "id": "2f2e3ad1-b57b-45d2-8f0b-cab25ca48fb7", - "status": "completed", - "date": "2011-05-05T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ead072ed-047e-45af-9100-0ac8f1dea0a3", - "resource": { - "resourceType": "Immunization", - "id": "ead072ed-047e-45af-9100-0ac8f1dea0a3", - "status": "completed", - "date": "2011-05-05T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:31109686-bf0b-410d-9920-1348ec1098fd" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e69956de-942c-4392-8ec3-b5e739d40d31", - "resource": { - "resourceType": "Claim", - "id": "e69956de-942c-4392-8ec3-b5e739d40d31", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc", - "resource": { - "resourceType": "Encounter", - "id": "2745c21c-b71f-4bba-9483-c3f5a54cd4fc", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2011-08-04T12:17:06-07:00", - "end": "2011-08-04T12:47:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:de78941e-6fe9-41c4-b476-e43c6b1e20eb", - "resource": { - "resourceType": "Condition", - "id": "de78941e-6fe9-41c4-b476-e43c6b1e20eb", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "dateRecorded": "2011-08-04", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53741008", - "display": "Coronary Heart Disease" - } - ], - "text": "Coronary Heart Disease" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2011-08-04T12:17:06-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b1becaba-940c-4bee-a906-80a7a507576f", - "resource": { - "resourceType": "Observation", - "id": "b1becaba-940c-4bee-a906-80a7a507576f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 68.400, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:840de12e-a048-4777-ac9c-c4936d7a116a", - "resource": { - "resourceType": "Observation", - "id": "840de12e-a048-4777-ac9c-c4936d7a116a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e867d25e-924d-421e-bd24-582a42999c9c", - "resource": { - "resourceType": "Observation", - "id": "e867d25e-924d-421e-bd24-582a42999c9c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 8.8000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:880dcbf4-d70a-4d8e-9e09-8ff3ba08228f", - "resource": { - "resourceType": "Observation", - "id": "880dcbf4-d70a-4d8e-9e09-8ff3ba08228f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 84.866, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:03e0b34c-edc1-4c7c-9649-c92b1aea0751", - "resource": { - "resourceType": "Observation", - "id": "03e0b34c-edc1-4c7c-9649-c92b1aea0751", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 43.830, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:62f0376a-f867-4808-b449-ecec3d0cb5fd", - "resource": { - "resourceType": "Observation", - "id": "62f0376a-f867-4808-b449-ecec3d0cb5fd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbb170a3-0b9a-4ec4-9d08-b9877ce5c654", - "resource": { - "resourceType": "Observation", - "id": "bbb170a3-0b9a-4ec4-9d08-b9877ce5c654", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 94, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f8701862-10d7-4ba9-8e86-9fdf26d0d736", - "resource": { - "resourceType": "Observation", - "id": "f8701862-10d7-4ba9-8e86-9fdf26d0d736", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c0e9599-cf99-4fe6-9e04-2da95998ab83", - "resource": { - "resourceType": "Observation", - "id": "5c0e9599-cf99-4fe6-9e04-2da95998ab83", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "effectiveDateTime": "2011-08-04T12:17:06-07:00", - "issued": "2011-08-04T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dee0f682-2456-4954-a845-a4a42c043469", - "resource": { - "resourceType": "Procedure", - "id": "dee0f682-2456-4954-a845-a4a42c043469", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2011-08-04T12:17:06-07:00", - "end": "2011-08-04T12:32:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a0d6efed-0215-4fcf-bafe-1580e606c135", - "resource": { - "resourceType": "MedicationOrder", - "id": "a0d6efed-0215-4fcf-bafe-1580e606c135", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:6b4dc1e0-7cbe-40e4-952c-37fc24919f1c", - "resource": { - "resourceType": "Claim", - "id": "6b4dc1e0-7cbe-40e4-952c-37fc24919f1c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a0d6efed-0215-4fcf-bafe-1580e606c135" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4559ce59-f77f-4935-9f38-7237e231a096", - "resource": { - "resourceType": "MedicationOrder", - "id": "4559ce59-f77f-4935-9f38-7237e231a096", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:2be067d2-2d80-4f98-b4e9-e7e70bc9d04a", - "resource": { - "resourceType": "Claim", - "id": "2be067d2-2d80-4f98-b4e9-e7e70bc9d04a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:4559ce59-f77f-4935-9f38-7237e231a096" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b85c572e-5bfa-427d-95db-4b588c360d10", - "resource": { - "resourceType": "MedicationOrder", - "id": "b85c572e-5bfa-427d-95db-4b588c360d10", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:a3eaaf3e-3b6b-4a1a-a726-c38ede34a454", - "resource": { - "resourceType": "Claim", - "id": "a3eaaf3e-3b6b-4a1a-a726-c38ede34a454", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:b85c572e-5bfa-427d-95db-4b588c360d10" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:55dd11b1-20d5-476a-a492-eacd0a925069", - "resource": { - "resourceType": "MedicationOrder", - "id": "55dd11b1-20d5-476a-a492-eacd0a925069", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:132079e0-2aa9-4267-855e-62a9127ed284", - "resource": { - "resourceType": "Claim", - "id": "132079e0-2aa9-4267-855e-62a9127ed284", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:55dd11b1-20d5-476a-a492-eacd0a925069" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:92eb4cd0-be66-4476-acc6-0b88a7624555", - "resource": { - "resourceType": "MedicationOrder", - "id": "92eb4cd0-be66-4476-acc6-0b88a7624555", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:cd469123-20ff-4ce0-9382-69aa82ae7f52", - "resource": { - "resourceType": "Claim", - "id": "cd469123-20ff-4ce0-9382-69aa82ae7f52", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:92eb4cd0-be66-4476-acc6-0b88a7624555" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ac2d62f3-484f-4762-96ff-a1bf61bafbf5", - "resource": { - "resourceType": "MedicationOrder", - "id": "ac2d62f3-484f-4762-96ff-a1bf61bafbf5", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:23cec18d-f254-4d89-80d3-12c19aaabeca", - "resource": { - "resourceType": "Claim", - "id": "23cec18d-f254-4d89-80d3-12c19aaabeca", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:ac2d62f3-484f-4762-96ff-a1bf61bafbf5" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:053ac539-d5c1-4873-b76c-f6258ee3d6aa", - "resource": { - "resourceType": "MedicationOrder", - "id": "053ac539-d5c1-4873-b76c-f6258ee3d6aa", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:14159c4f-9943-4128-a1cf-ea08b1d7713e", - "resource": { - "resourceType": "Claim", - "id": "14159c4f-9943-4128-a1cf-ea08b1d7713e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:053ac539-d5c1-4873-b76c-f6258ee3d6aa" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:16f34136-9f5a-412d-88e0-a63f927b7307", - "resource": { - "resourceType": "MedicationOrder", - "id": "16f34136-9f5a-412d-88e0-a63f927b7307", - "dateWritten": "2011-08-04T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:79ee24b2-23ee-4ee4-af1d-a3d5b592e129", - "resource": { - "resourceType": "Claim", - "id": "79ee24b2-23ee-4ee4-af1d-a3d5b592e129", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:16f34136-9f5a-412d-88e0-a63f927b7307" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bf4f03f9-4995-4d57-9bad-c68805e10e56", - "resource": { - "resourceType": "Immunization", - "id": "bf4f03f9-4995-4d57-9bad-c68805e10e56", - "status": "completed", - "date": "2011-08-04T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:214e3558-1979-4783-98b0-3325539a3d6c", - "resource": { - "resourceType": "Immunization", - "id": "214e3558-1979-4783-98b0-3325539a3d6c", - "status": "completed", - "date": "2011-08-04T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:9a0f2346-7a9f-4857-9bbf-88e1d8c8e8e3", - "resource": { - "resourceType": "Immunization", - "id": "9a0f2346-7a9f-4857-9bbf-88e1d8c8e8e3", - "status": "completed", - "date": "2011-08-04T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2006e537-92e8-4850-9cee-a26da05d7b98", - "resource": { - "resourceType": "Immunization", - "id": "2006e537-92e8-4850-9cee-a26da05d7b98", - "status": "completed", - "date": "2011-08-04T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8dbba982-dd89-4979-a288-5e9161502b1e", - "resource": { - "resourceType": "Immunization", - "id": "8dbba982-dd89-4979-a288-5e9161502b1e", - "status": "completed", - "date": "2011-08-04T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2745c21c-b71f-4bba-9483-c3f5a54cd4fc" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:39f0d03a-9e58-49c6-a7a3-470ccbe8c1cd", - "resource": { - "resourceType": "Claim", - "id": "39f0d03a-9e58-49c6-a7a3-470ccbe8c1cd", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "53741008", - "display": "Coronary Heart Disease" - } - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 8, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 699.53, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287", - "resource": { - "resourceType": "Encounter", - "id": "0d6730ed-1bc4-40ca-9483-9b46ce6f1287", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2011-11-03T12:17:06-07:00", - "end": "2011-11-03T12:32:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9a8ab457-866c-4253-82f3-895d6cb0642d", - "resource": { - "resourceType": "Observation", - "id": "9a8ab457-866c-4253-82f3-895d6cb0642d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 72.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0f54d30e-add4-4071-a554-bff23ea00150", - "resource": { - "resourceType": "Observation", - "id": "0f54d30e-add4-4071-a554-bff23ea00150", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35ff6076-2462-48a0-a8b0-be407d1fdb9a", - "resource": { - "resourceType": "Observation", - "id": "35ff6076-2462-48a0-a8b0-be407d1fdb9a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 9.9000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecc38ff1-69be-4d09-bdf6-201aa0ad43ae", - "resource": { - "resourceType": "Observation", - "id": "ecc38ff1-69be-4d09-bdf6-201aa0ad43ae", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 86.451, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2962ef3f-d9d0-4b4c-a89e-f2f27b7daeb8", - "resource": { - "resourceType": "Observation", - "id": "2962ef3f-d9d0-4b4c-a89e-f2f27b7daeb8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 44.980, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e75e6156-ee15-4c17-ace7-efd2328f3066", - "resource": { - "resourceType": "Observation", - "id": "e75e6156-ee15-4c17-ace7-efd2328f3066", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12d4913a-9e64-44ff-87b3-16fbf9c9de94", - "resource": { - "resourceType": "Observation", - "id": "12d4913a-9e64-44ff-87b3-16fbf9c9de94", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 80, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b5765ee-4a0f-4dea-8c74-ffd1bd14a474", - "resource": { - "resourceType": "Observation", - "id": "2b5765ee-4a0f-4dea-8c74-ffd1bd14a474", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a402273a-5722-468a-8c9b-4a7db2182a85", - "resource": { - "resourceType": "Observation", - "id": "a402273a-5722-468a-8c9b-4a7db2182a85", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "effectiveDateTime": "2011-11-03T12:17:06-07:00", - "issued": "2011-11-03T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e787d579-dc45-475b-bf2e-cac9ce29366e", - "resource": { - "resourceType": "MedicationOrder", - "id": "e787d579-dc45-475b-bf2e-cac9ce29366e", - "dateWritten": "2011-11-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:a08297f4-01b8-481c-96e2-2f61c11b7ec7", - "resource": { - "resourceType": "Claim", - "id": "a08297f4-01b8-481c-96e2-2f61c11b7ec7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:e787d579-dc45-475b-bf2e-cac9ce29366e" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9d8ebca3-9068-4768-b34a-a73776cbdad0", - "resource": { - "resourceType": "MedicationOrder", - "id": "9d8ebca3-9068-4768-b34a-a73776cbdad0", - "dateWritten": "2011-11-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:b69bb887-fefe-4cfe-a0ea-7a58fed29f68", - "resource": { - "resourceType": "Claim", - "id": "b69bb887-fefe-4cfe-a0ea-7a58fed29f68", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:9d8ebca3-9068-4768-b34a-a73776cbdad0" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4675571a-bfbe-42ef-b661-ee543a680571", - "resource": { - "resourceType": "MedicationOrder", - "id": "4675571a-bfbe-42ef-b661-ee543a680571", - "dateWritten": "2011-11-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:cf0be6be-3e04-463f-9a08-a56b2ef4b530", - "resource": { - "resourceType": "Claim", - "id": "cf0be6be-3e04-463f-9a08-a56b2ef4b530", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:4675571a-bfbe-42ef-b661-ee543a680571" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8556eace-d21f-4635-b8ef-e2820f5c70ce", - "resource": { - "resourceType": "MedicationOrder", - "id": "8556eace-d21f-4635-b8ef-e2820f5c70ce", - "dateWritten": "2011-11-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:0d6730ed-1bc4-40ca-9483-9b46ce6f1287" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:65200abf-a80e-4ebc-9aa1-3a2a403826e9", - "resource": { - "resourceType": "Claim", - "id": "65200abf-a80e-4ebc-9aa1-3a2a403826e9", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:8556eace-d21f-4635-b8ef-e2820f5c70ce" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:36996e46-a7f8-4495-a000-2d1e284e2437", - "resource": { - "resourceType": "Claim", - "id": "36996e46-a7f8-4495-a000-2d1e284e2437", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3", - "resource": { - "resourceType": "Encounter", - "id": "7ff08b3d-8101-4e74-b528-f4709ad7bbe3", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2012-02-02T11:17:06-08:00", - "end": "2012-02-02T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:530cce89-b948-4d6c-95e1-bc8c22afcb1a", - "resource": { - "resourceType": "Observation", - "id": "530cce89-b948-4d6c-95e1-bc8c22afcb1a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 75.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:abdf4c7a-865f-4a02-853d-3c9274aa60e0", - "resource": { - "resourceType": "Observation", - "id": "abdf4c7a-865f-4a02-853d-3c9274aa60e0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b4957d1-01ac-4ed8-81ae-13af964d44d2", - "resource": { - "resourceType": "Observation", - "id": "6b4957d1-01ac-4ed8-81ae-13af964d44d2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 10.700, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6858eb67-2ced-4085-b704-a61c8b1d0925", - "resource": { - "resourceType": "Observation", - "id": "6858eb67-2ced-4085-b704-a61c8b1d0925", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 80.884, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff0a45c8-11c5-4f03-a277-304dc35a9c85", - "resource": { - "resourceType": "Observation", - "id": "ff0a45c8-11c5-4f03-a277-304dc35a9c85", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 45.810, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6c8fae00-80b4-4539-a01f-0ec48a6fae91", - "resource": { - "resourceType": "Observation", - "id": "6c8fae00-80b4-4539-a01f-0ec48a6fae91", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 71, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 107, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d97c313f-17ea-40b5-b4af-3328e1f9fa1a", - "resource": { - "resourceType": "Observation", - "id": "d97c313f-17ea-40b5-b4af-3328e1f9fa1a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 95, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d44864cf-98b0-42c4-a9cd-37d9b423f5cc", - "resource": { - "resourceType": "Observation", - "id": "d44864cf-98b0-42c4-a9cd-37d9b423f5cc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a77d622e-04a5-414c-9e07-075093068075", - "resource": { - "resourceType": "Observation", - "id": "a77d622e-04a5-414c-9e07-075093068075", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "effectiveDateTime": "2012-02-02T11:17:06-08:00", - "issued": "2012-02-02T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:392921a7-98f1-43df-be6d-ec45da239376", - "resource": { - "resourceType": "Procedure", - "id": "392921a7-98f1-43df-be6d-ec45da239376", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2012-02-02T11:17:06-08:00", - "end": "2012-02-02T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:204d30f2-01a4-4347-be9c-8f1cfb77b251", - "resource": { - "resourceType": "MedicationOrder", - "id": "204d30f2-01a4-4347-be9c-8f1cfb77b251", - "dateWritten": "2012-02-02T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:4f42779d-89c4-4ae4-bb4e-b040460aa4a2", - "resource": { - "resourceType": "Claim", - "id": "4f42779d-89c4-4ae4-bb4e-b040460aa4a2", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:204d30f2-01a4-4347-be9c-8f1cfb77b251" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:412738a7-1b3d-4727-b390-f24369bbdb88", - "resource": { - "resourceType": "MedicationOrder", - "id": "412738a7-1b3d-4727-b390-f24369bbdb88", - "dateWritten": "2012-02-02T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:1bb60d5d-bf69-4a63-b906-f01b74d8bb99", - "resource": { - "resourceType": "Claim", - "id": "1bb60d5d-bf69-4a63-b906-f01b74d8bb99", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:412738a7-1b3d-4727-b390-f24369bbdb88" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d69c2510-34be-495f-93da-3cb81620cf57", - "resource": { - "resourceType": "MedicationOrder", - "id": "d69c2510-34be-495f-93da-3cb81620cf57", - "dateWritten": "2012-02-02T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:ff381297-d1a8-4a39-865c-60b2f828177c", - "resource": { - "resourceType": "Claim", - "id": "ff381297-d1a8-4a39-865c-60b2f828177c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:d69c2510-34be-495f-93da-3cb81620cf57" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c16b38ad-e77e-4782-924c-e4e92d92b0a1", - "resource": { - "resourceType": "MedicationOrder", - "id": "c16b38ad-e77e-4782-924c-e4e92d92b0a1", - "dateWritten": "2012-02-02T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:7647de31-3286-4bf2-a74d-43aacdef3819", - "resource": { - "resourceType": "Claim", - "id": "7647de31-3286-4bf2-a74d-43aacdef3819", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:c16b38ad-e77e-4782-924c-e4e92d92b0a1" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ad6fb0e5-49d8-4653-a76d-bbf3457031bd", - "resource": { - "resourceType": "Immunization", - "id": "ad6fb0e5-49d8-4653-a76d-bbf3457031bd", - "status": "completed", - "date": "2012-02-02T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a513a582-d4c1-4212-a145-6ce2851137db", - "resource": { - "resourceType": "Immunization", - "id": "a513a582-d4c1-4212-a145-6ce2851137db", - "status": "completed", - "date": "2012-02-02T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f5a25186-8040-4bcf-a2c0-2f45e07b168e", - "resource": { - "resourceType": "Immunization", - "id": "f5a25186-8040-4bcf-a2c0-2f45e07b168e", - "status": "completed", - "date": "2012-02-02T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e1e69273-8ab2-4504-9f36-391d9486a71c", - "resource": { - "resourceType": "Immunization", - "id": "e1e69273-8ab2-4504-9f36-391d9486a71c", - "status": "completed", - "date": "2012-02-02T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2ba70002-3750-4fd3-904a-967cf3abb1fe", - "resource": { - "resourceType": "Immunization", - "id": "2ba70002-3750-4fd3-904a-967cf3abb1fe", - "status": "completed", - "date": "2012-02-02T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7ff08b3d-8101-4e74-b528-f4709ad7bbe3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:49c50f34-3a7c-4577-832c-b971599dbe6e", - "resource": { - "resourceType": "Claim", - "id": "49c50f34-3a7c-4577-832c-b971599dbe6e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 460.25, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6", - "resource": { - "resourceType": "Encounter", - "id": "465b8554-145a-41bf-9423-dc26a4d195f6", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2012-05-03T12:17:06-07:00", - "end": "2012-05-03T12:32:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:53e25dfb-8a4b-47d5-9ddc-746fcc5862c8", - "resource": { - "resourceType": "Observation", - "id": "53e25dfb-8a4b-47d5-9ddc-746fcc5862c8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 78.400, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:da6acbbb-090d-460c-a2b9-04befb1928ef", - "resource": { - "resourceType": "Observation", - "id": "da6acbbb-090d-460c-a2b9-04befb1928ef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:56767a3d-01dd-4554-99c4-e1c4afe512d2", - "resource": { - "resourceType": "Observation", - "id": "56767a3d-01dd-4554-99c4-e1c4afe512d2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 11.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4451c32-7677-4954-a27d-578eb5a54262", - "resource": { - "resourceType": "Observation", - "id": "c4451c32-7677-4954-a27d-578eb5a54262", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 84.041, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3be791f-2004-4752-a42a-e58ed13f913b", - "resource": { - "resourceType": "Observation", - "id": "b3be791f-2004-4752-a42a-e58ed13f913b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 46.420, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2e7825de-e0f7-4d15-a6ee-03df392572e2", - "resource": { - "resourceType": "Observation", - "id": "2e7825de-e0f7-4d15-a6ee-03df392572e2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e5aa27b2-4c33-4047-974d-a70a917aa5d3", - "resource": { - "resourceType": "Observation", - "id": "e5aa27b2-4c33-4047-974d-a70a917aa5d3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a00c22c3-e18c-4ad4-973e-00daf5adae45", - "resource": { - "resourceType": "Observation", - "id": "a00c22c3-e18c-4ad4-973e-00daf5adae45", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4dd0f81e-254c-4d1a-b29f-b90e3cd78b10", - "resource": { - "resourceType": "Observation", - "id": "4dd0f81e-254c-4d1a-b29f-b90e3cd78b10", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "effectiveDateTime": "2012-05-03T12:17:06-07:00", - "issued": "2012-05-03T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84f455d6-c625-4ce2-8130-7a6e94ec2d14", - "resource": { - "resourceType": "MedicationOrder", - "id": "84f455d6-c625-4ce2-8130-7a6e94ec2d14", - "dateWritten": "2012-05-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:003b3b0b-ec0b-4962-8f68-e5af7754dd3a", - "resource": { - "resourceType": "Claim", - "id": "003b3b0b-ec0b-4962-8f68-e5af7754dd3a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:84f455d6-c625-4ce2-8130-7a6e94ec2d14" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:38e77308-0d4a-470f-9713-9492eff388af", - "resource": { - "resourceType": "MedicationOrder", - "id": "38e77308-0d4a-470f-9713-9492eff388af", - "dateWritten": "2012-05-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:32d3acf9-c8e2-401f-b9f9-f1f192e5f8ab", - "resource": { - "resourceType": "Claim", - "id": "32d3acf9-c8e2-401f-b9f9-f1f192e5f8ab", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:38e77308-0d4a-470f-9713-9492eff388af" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a7086955-dd8a-4d67-ab5c-2c21475e3d4d", - "resource": { - "resourceType": "MedicationOrder", - "id": "a7086955-dd8a-4d67-ab5c-2c21475e3d4d", - "dateWritten": "2012-05-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:ff1a3820-04c3-4b6e-ad11-67104b17fe43", - "resource": { - "resourceType": "Claim", - "id": "ff1a3820-04c3-4b6e-ad11-67104b17fe43", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a7086955-dd8a-4d67-ab5c-2c21475e3d4d" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:04c17a70-6744-459f-9d96-a0b4d608f50b", - "resource": { - "resourceType": "MedicationOrder", - "id": "04c17a70-6744-459f-9d96-a0b4d608f50b", - "dateWritten": "2012-05-03T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:8fcb9d18-e96f-4627-81f1-8ed9b32ad6ee", - "resource": { - "resourceType": "Claim", - "id": "8fcb9d18-e96f-4627-81f1-8ed9b32ad6ee", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:04c17a70-6744-459f-9d96-a0b4d608f50b" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c010f27b-50ee-4280-b90c-e26a1c9ce671", - "resource": { - "resourceType": "Immunization", - "id": "c010f27b-50ee-4280-b90c-e26a1c9ce671", - "status": "completed", - "date": "2012-05-03T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:465b8554-145a-41bf-9423-dc26a4d195f6" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:28f8607b-3bab-478a-873b-31686e483fa0", - "resource": { - "resourceType": "Claim", - "id": "28f8607b-3bab-478a-873b-31686e483fa0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d", - "resource": { - "resourceType": "Encounter", - "id": "8a2ea952-1c47-4cf8-a748-d14ebfb5144d", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2012-11-01T12:17:06-07:00", - "end": "2012-11-01T12:47:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:40d0aeaf-90f0-4b5e-b8bc-540dafa6e58f", - "resource": { - "resourceType": "Observation", - "id": "40d0aeaf-90f0-4b5e-b8bc-540dafa6e58f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 83.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b66440b-c784-425d-911c-0a05df513978", - "resource": { - "resourceType": "Observation", - "id": "0b66440b-c784-425d-911c-0a05df513978", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df8e4146-01f1-4051-b074-300f8a6ec1a4", - "resource": { - "resourceType": "Observation", - "id": "df8e4146-01f1-4051-b074-300f8a6ec1a4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 12.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab3b7842-6fac-4402-872a-6911bc343fe3", - "resource": { - "resourceType": "Observation", - "id": "ab3b7842-6fac-4402-872a-6911bc343fe3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 67.104, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:901daf0f-cde0-46f3-8aea-37e9e57017f0", - "resource": { - "resourceType": "Observation", - "id": "901daf0f-cde0-46f3-8aea-37e9e57017f0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 47.25, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e69b070-974f-40f3-92b7-ab59cc947574", - "resource": { - "resourceType": "Observation", - "id": "4e69b070-974f-40f3-92b7-ab59cc947574", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d467c45-1e68-49d9-ae72-e57f296ccc31", - "resource": { - "resourceType": "Observation", - "id": "0d467c45-1e68-49d9-ae72-e57f296ccc31", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 99, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:07ba669b-ba3f-47e8-9933-77887f3a4bf8", - "resource": { - "resourceType": "Observation", - "id": "07ba669b-ba3f-47e8-9933-77887f3a4bf8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8087197c-1c20-4c51-bea7-2d6731d75bb4", - "resource": { - "resourceType": "Observation", - "id": "8087197c-1c20-4c51-bea7-2d6731d75bb4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "effectiveDateTime": "2012-11-01T12:17:06-07:00", - "issued": "2012-11-01T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ead65c5-4b51-4587-b3b9-b984ecd58370", - "resource": { - "resourceType": "Procedure", - "id": "0ead65c5-4b51-4587-b3b9-b984ecd58370", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2012-11-01T12:17:06-07:00", - "end": "2012-11-01T12:32:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3", - "resource": { - "resourceType": "MedicationOrder", - "id": "3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3", - "dateWritten": "2012-11-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:45dfb83d-2e0e-4c51-94c8-194abd11b1cf", - "resource": { - "resourceType": "Claim", - "id": "45dfb83d-2e0e-4c51-94c8-194abd11b1cf", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:3e75ea2a-7b4e-4dac-a6e0-634a0920b8f3" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:af23f926-a8d4-4ae7-9dd3-f4f340e3066b", - "resource": { - "resourceType": "MedicationOrder", - "id": "af23f926-a8d4-4ae7-9dd3-f4f340e3066b", - "dateWritten": "2012-11-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:843584f6-fb45-4135-b3fe-829c961c418d", - "resource": { - "resourceType": "Claim", - "id": "843584f6-fb45-4135-b3fe-829c961c418d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:af23f926-a8d4-4ae7-9dd3-f4f340e3066b" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8b606c95-ef85-4dd4-82f6-dcb37e8c2b23", - "resource": { - "resourceType": "MedicationOrder", - "id": "8b606c95-ef85-4dd4-82f6-dcb37e8c2b23", - "dateWritten": "2012-11-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:48297181-250d-4925-ae64-715d3d3342ab", - "resource": { - "resourceType": "Claim", - "id": "48297181-250d-4925-ae64-715d3d3342ab", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:8b606c95-ef85-4dd4-82f6-dcb37e8c2b23" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a98058c1-7dbd-412c-bb9c-4c61123bcdb4", - "resource": { - "resourceType": "MedicationOrder", - "id": "a98058c1-7dbd-412c-bb9c-4c61123bcdb4", - "dateWritten": "2012-11-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:6d3ad287-cb02-4125-ad4d-3467e19e1afb", - "resource": { - "resourceType": "Claim", - "id": "6d3ad287-cb02-4125-ad4d-3467e19e1afb", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a98058c1-7dbd-412c-bb9c-4c61123bcdb4" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:933b1d82-9779-4e72-9de4-4940b51e3a12", - "resource": { - "resourceType": "Immunization", - "id": "933b1d82-9779-4e72-9de4-4940b51e3a12", - "status": "completed", - "date": "2012-11-01T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:8a2ea952-1c47-4cf8-a748-d14ebfb5144d" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8c02c2a5-1e8c-4637-bd48-3c1eba65e5ef", - "resource": { - "resourceType": "Claim", - "id": "8c02c2a5-1e8c-4637-bd48-3c1eba65e5ef", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 817.84, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9", - "resource": { - "resourceType": "Encounter", - "id": "86fce1d4-25e1-4915-b644-9cfdbf32feb9", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2013-05-02T12:17:06-07:00", - "end": "2013-05-02T12:32:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b0563b7c-f103-4cde-a8bb-bacf07a87bf0", - "resource": { - "resourceType": "Observation", - "id": "b0563b7c-f103-4cde-a8bb-bacf07a87bf0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 87.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c3990c74-0493-4fc5-98ed-60eb5598b655", - "resource": { - "resourceType": "Observation", - "id": "c3990c74-0493-4fc5-98ed-60eb5598b655", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e08634b3-4e86-4ae4-9aa2-093c790d6f15", - "resource": { - "resourceType": "Observation", - "id": "e08634b3-4e86-4ae4-9aa2-093c790d6f15", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 13, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:39038ef5-60d5-446b-a2b6-b60d579c4453", - "resource": { - "resourceType": "Observation", - "id": "39038ef5-60d5-446b-a2b6-b60d579c4453", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 70.606, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7a7f1b5b-5fa2-41f5-b709-8e193b4e71a7", - "resource": { - "resourceType": "Observation", - "id": "7a7f1b5b-5fa2-41f5-b709-8e193b4e71a7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 47.760, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3ddea093-5118-46c0-bfc0-696301172acf", - "resource": { - "resourceType": "Observation", - "id": "3ddea093-5118-46c0-bfc0-696301172acf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 17.150, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5035e3b-e2a1-4e60-a1c3-825d9350230d", - "resource": { - "resourceType": "Observation", - "id": "a5035e3b-e2a1-4e60-a1c3-825d9350230d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 74.062, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e2324eb-9a70-414a-8daf-83a9a6b31566", - "resource": { - "resourceType": "Observation", - "id": "5e2324eb-9a70-414a-8daf-83a9a6b31566", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 128, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8755f73-ec5e-4c87-b6f6-388b8ad4728b", - "resource": { - "resourceType": "Observation", - "id": "e8755f73-ec5e-4c87-b6f6-388b8ad4728b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 73, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b588520-047b-44d2-be15-8a7b79d3bee2", - "resource": { - "resourceType": "Observation", - "id": "0b588520-047b-44d2-be15-8a7b79d3bee2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68a9b083-74f4-4522-a57e-6a37aa19539c", - "resource": { - "resourceType": "Observation", - "id": "68a9b083-74f4-4522-a57e-6a37aa19539c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "effectiveDateTime": "2013-05-02T12:17:06-07:00", - "issued": "2013-05-02T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:907ae086-6c1b-4028-a658-ac1ec4bb5dd6", - "resource": { - "resourceType": "MedicationOrder", - "id": "907ae086-6c1b-4028-a658-ac1ec4bb5dd6", - "dateWritten": "2013-05-02T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:56958178-94cc-48dc-b936-5183ab1d12d0", - "resource": { - "resourceType": "Claim", - "id": "56958178-94cc-48dc-b936-5183ab1d12d0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:907ae086-6c1b-4028-a658-ac1ec4bb5dd6" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3dfe1813-2a01-4590-8087-92ad9f9bb1d0", - "resource": { - "resourceType": "MedicationOrder", - "id": "3dfe1813-2a01-4590-8087-92ad9f9bb1d0", - "dateWritten": "2013-05-02T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:dd41427a-e723-414d-a53b-c9138e559daa", - "resource": { - "resourceType": "Claim", - "id": "dd41427a-e723-414d-a53b-c9138e559daa", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:3dfe1813-2a01-4590-8087-92ad9f9bb1d0" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:30cf44a8-f2bb-47c4-a946-063dc4c69e83", - "resource": { - "resourceType": "MedicationOrder", - "id": "30cf44a8-f2bb-47c4-a946-063dc4c69e83", - "dateWritten": "2013-05-02T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:e9fb7e56-7606-4838-9899-8f40dee4b729", - "resource": { - "resourceType": "Claim", - "id": "e9fb7e56-7606-4838-9899-8f40dee4b729", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:30cf44a8-f2bb-47c4-a946-063dc4c69e83" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f423073c-4b1c-4d42-a7ac-f8878e4ad24b", - "resource": { - "resourceType": "MedicationOrder", - "id": "f423073c-4b1c-4d42-a7ac-f8878e4ad24b", - "dateWritten": "2013-05-02T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:1dbfdcf9-280c-4b5d-a24a-3a0d0c8808e1", - "resource": { - "resourceType": "Claim", - "id": "1dbfdcf9-280c-4b5d-a24a-3a0d0c8808e1", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:f423073c-4b1c-4d42-a7ac-f8878e4ad24b" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:137807f4-5555-4a52-93bb-81307f85b521", - "resource": { - "resourceType": "Immunization", - "id": "137807f4-5555-4a52-93bb-81307f85b521", - "status": "completed", - "date": "2013-05-02T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:86fce1d4-25e1-4915-b644-9cfdbf32feb9" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4e717831-f892-44ad-8355-3a6e05eb8618", - "resource": { - "resourceType": "Claim", - "id": "4e717831-f892-44ad-8355-3a6e05eb8618", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71", - "resource": { - "resourceType": "Encounter", - "id": "29673ff6-9827-48ce-9bcd-9fa3e0e50f71", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2013-10-31T12:17:06-07:00", - "end": "2013-10-31T12:47:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bff987d7-f74c-4506-8ca8-6940d5fb05a4", - "resource": { - "resourceType": "Observation", - "id": "bff987d7-f74c-4506-8ca8-6940d5fb05a4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 91.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1359ec09-da6d-43d5-9477-13d2076b6328", - "resource": { - "resourceType": "Observation", - "id": "1359ec09-da6d-43d5-9477-13d2076b6328", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:54507301-4101-4cf8-97e2-abac21828a63", - "resource": { - "resourceType": "Observation", - "id": "54507301-4101-4cf8-97e2-abac21828a63", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 13.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3ac3ae97-437b-4f23-bebe-157dea93f661", - "resource": { - "resourceType": "Observation", - "id": "3ac3ae97-437b-4f23-bebe-157dea93f661", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 68.648, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c3956f9-77ff-49c5-8ef4-98effb797ab4", - "resource": { - "resourceType": "Observation", - "id": "2c3956f9-77ff-49c5-8ef4-98effb797ab4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 48.070, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:837c31f3-6766-4098-87ed-6e4b9d87d928", - "resource": { - "resourceType": "Observation", - "id": "837c31f3-6766-4098-87ed-6e4b9d87d928", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 16.780, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fd029216-5fc1-4500-9912-3c1a2e78825d", - "resource": { - "resourceType": "Observation", - "id": "fd029216-5fc1-4500-9912-3c1a2e78825d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 72.591, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2588f404-45f3-4789-8a3c-08500cad701e", - "resource": { - "resourceType": "Observation", - "id": "2588f404-45f3-4789-8a3c-08500cad701e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 123, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0622c5e-ea7d-453b-9a46-625e39dc7a54", - "resource": { - "resourceType": "Observation", - "id": "b0622c5e-ea7d-453b-9a46-625e39dc7a54", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 88, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d85956e-6682-4d4e-b84e-bc63e0b73b6b", - "resource": { - "resourceType": "Observation", - "id": "7d85956e-6682-4d4e-b84e-bc63e0b73b6b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff65eaaa-ed71-4863-9c32-defa9bbfa409", - "resource": { - "resourceType": "Observation", - "id": "ff65eaaa-ed71-4863-9c32-defa9bbfa409", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "effectiveDateTime": "2013-10-31T12:17:06-07:00", - "issued": "2013-10-31T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b7100ec-3ce9-4da7-b21f-80b2ed537330", - "resource": { - "resourceType": "Procedure", - "id": "6b7100ec-3ce9-4da7-b21f-80b2ed537330", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2013-10-31T12:17:06-07:00", - "end": "2013-10-31T12:32:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:63beaa22-0d81-4a73-97ea-d67ece4f01df", - "resource": { - "resourceType": "MedicationOrder", - "id": "63beaa22-0d81-4a73-97ea-d67ece4f01df", - "dateWritten": "2013-10-31T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:edd87983-8719-468c-aa41-ee42aecdd09c", - "resource": { - "resourceType": "Claim", - "id": "edd87983-8719-468c-aa41-ee42aecdd09c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:63beaa22-0d81-4a73-97ea-d67ece4f01df" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1aadd4b8-6c66-430f-83c6-7cd3c60c7f90", - "resource": { - "resourceType": "MedicationOrder", - "id": "1aadd4b8-6c66-430f-83c6-7cd3c60c7f90", - "dateWritten": "2013-10-31T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:09fe6cac-1c69-46e2-8f6c-4d02ffe0ef66", - "resource": { - "resourceType": "Claim", - "id": "09fe6cac-1c69-46e2-8f6c-4d02ffe0ef66", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:1aadd4b8-6c66-430f-83c6-7cd3c60c7f90" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:678ad620-5b9a-4352-9f23-29cf9c3a912c", - "resource": { - "resourceType": "MedicationOrder", - "id": "678ad620-5b9a-4352-9f23-29cf9c3a912c", - "dateWritten": "2013-10-31T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:b190063c-e1aa-414b-b243-64d73f79e1af", - "resource": { - "resourceType": "Claim", - "id": "b190063c-e1aa-414b-b243-64d73f79e1af", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:678ad620-5b9a-4352-9f23-29cf9c3a912c" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:17a05c27-33da-44f9-996a-4430cd0d7e31", - "resource": { - "resourceType": "MedicationOrder", - "id": "17a05c27-33da-44f9-996a-4430cd0d7e31", - "dateWritten": "2013-10-31T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:756fbfe9-c7d7-4f1d-bf5b-a2c9d4df78c4", - "resource": { - "resourceType": "Claim", - "id": "756fbfe9-c7d7-4f1d-bf5b-a2c9d4df78c4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:17a05c27-33da-44f9-996a-4430cd0d7e31" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:92917a82-cb51-4098-a4b3-ae37939a35dd", - "resource": { - "resourceType": "Immunization", - "id": "92917a82-cb51-4098-a4b3-ae37939a35dd", - "status": "completed", - "date": "2013-10-31T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:29673ff6-9827-48ce-9bcd-9fa3e0e50f71" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:24941710-6135-4b5b-aade-c5eeffb63843", - "resource": { - "resourceType": "Claim", - "id": "24941710-6135-4b5b-aade-c5eeffb63843", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 562.93, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3", - "resource": { - "resourceType": "Encounter", - "id": "fe8a9231-bcca-4236-88ca-d37957944cd3", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2014-05-01T12:17:06-07:00", - "end": "2014-05-01T12:32:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4f50079a-1238-4d00-949d-b79b114dd934", - "resource": { - "resourceType": "Observation", - "id": "4f50079a-1238-4d00-949d-b79b114dd934", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 94.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bd68506-8b8f-48ad-a313-2bcb8a0fc968", - "resource": { - "resourceType": "Observation", - "id": "7bd68506-8b8f-48ad-a313-2bcb8a0fc968", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ce72f5ed-e116-4da6-a5b0-c5ab069c06ed", - "resource": { - "resourceType": "Observation", - "id": "ce72f5ed-e116-4da6-a5b0-c5ab069c06ed", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 14.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b076ad63-5db5-4f9b-afeb-46750f0ac4ab", - "resource": { - "resourceType": "Observation", - "id": "b076ad63-5db5-4f9b-afeb-46750f0ac4ab", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 67.192, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d7400b50-b800-4ac9-9ba4-c7a7bb0b4941", - "resource": { - "resourceType": "Observation", - "id": "d7400b50-b800-4ac9-9ba4-c7a7bb0b4941", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 48.090, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68f8d965-4ec2-4e16-b138-606ed985d882", - "resource": { - "resourceType": "Observation", - "id": "68f8d965-4ec2-4e16-b138-606ed985d882", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 16.650, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b0217b1-bc86-4579-ae82-a0313b1c0732", - "resource": { - "resourceType": "Observation", - "id": "4b0217b1-bc86-4579-ae82-a0313b1c0732", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 75.283, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a48ea91d-7651-41d7-9e56-9a1359de3c4b", - "resource": { - "resourceType": "Observation", - "id": "a48ea91d-7651-41d7-9e56-9a1359de3c4b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 112, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6da220e0-be3a-4e05-8482-dd65c26d2a9e", - "resource": { - "resourceType": "Observation", - "id": "6da220e0-be3a-4e05-8482-dd65c26d2a9e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 80, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33d7a8b6-1941-4429-8ca9-f906a362ca95", - "resource": { - "resourceType": "Observation", - "id": "33d7a8b6-1941-4429-8ca9-f906a362ca95", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92858dce-3f4e-448b-8bfa-25af37db5a16", - "resource": { - "resourceType": "Observation", - "id": "92858dce-3f4e-448b-8bfa-25af37db5a16", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "effectiveDateTime": "2014-05-01T12:17:06-07:00", - "issued": "2014-05-01T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc", - "resource": { - "resourceType": "MedicationOrder", - "id": "dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc", - "dateWritten": "2014-05-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:86e00ca5-98a6-462a-953e-da70438ed2bc", - "resource": { - "resourceType": "Claim", - "id": "86e00ca5-98a6-462a-953e-da70438ed2bc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:dc9e01d5-0439-4d1d-bfb9-3bfc6a2459dc" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fd888874-0be5-4e2c-a572-aa50fcde16ba", - "resource": { - "resourceType": "MedicationOrder", - "id": "fd888874-0be5-4e2c-a572-aa50fcde16ba", - "dateWritten": "2014-05-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:f52a3344-b123-417e-90bb-b2229009db7c", - "resource": { - "resourceType": "Claim", - "id": "f52a3344-b123-417e-90bb-b2229009db7c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:fd888874-0be5-4e2c-a572-aa50fcde16ba" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0521e4ee-2112-4099-8d3e-119302e8a5bf", - "resource": { - "resourceType": "MedicationOrder", - "id": "0521e4ee-2112-4099-8d3e-119302e8a5bf", - "dateWritten": "2014-05-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:a41b7f4c-78e6-4078-9523-fb49108d10bf", - "resource": { - "resourceType": "Claim", - "id": "a41b7f4c-78e6-4078-9523-fb49108d10bf", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:0521e4ee-2112-4099-8d3e-119302e8a5bf" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:842bb8ba-e02d-4935-8fe5-3573b6c2836e", - "resource": { - "resourceType": "MedicationOrder", - "id": "842bb8ba-e02d-4935-8fe5-3573b6c2836e", - "dateWritten": "2014-05-01T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:fe8a9231-bcca-4236-88ca-d37957944cd3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:a16e708f-062b-4547-85a6-f8abb56c76fe", - "resource": { - "resourceType": "Claim", - "id": "a16e708f-062b-4547-85a6-f8abb56c76fe", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:842bb8ba-e02d-4935-8fe5-3573b6c2836e" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:10259cfd-2bf8-4a73-804a-1bf0bf5b6126", - "resource": { - "resourceType": "Claim", - "id": "10259cfd-2bf8-4a73-804a-1bf0bf5b6126", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4", - "resource": { - "resourceType": "Encounter", - "id": "6b54f746-9c2e-4a36-a384-59c755778ce4", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2014-10-30T12:17:06-07:00", - "end": "2014-10-30T12:47:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6a35034f-5616-4f06-a2d0-1c7ce2eedb9a", - "resource": { - "resourceType": "Observation", - "id": "6a35034f-5616-4f06-a2d0-1c7ce2eedb9a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 97.900, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:395b9a64-6a17-4a45-beab-955f0262aeb4", - "resource": { - "resourceType": "Observation", - "id": "395b9a64-6a17-4a45-beab-955f0262aeb4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:086a10f7-18dc-4b56-ae52-9a7871b9c258", - "resource": { - "resourceType": "Observation", - "id": "086a10f7-18dc-4b56-ae52-9a7871b9c258", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 15.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0018d3fe-276d-4f04-aad5-f17cf91331b1", - "resource": { - "resourceType": "Observation", - "id": "0018d3fe-276d-4f04-aad5-f17cf91331b1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 67.192, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83888cce-9797-4e11-9ad9-48f2bee938e3", - "resource": { - "resourceType": "Observation", - "id": "83888cce-9797-4e11-9ad9-48f2bee938e3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 48.090, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:414f6fb4-bdef-4458-9e4c-c8c7b5da92ef", - "resource": { - "resourceType": "Observation", - "id": "414f6fb4-bdef-4458-9e4c-c8c7b5da92ef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 16.610, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d64fc45b-4be9-43ba-b3ef-79ec46006e82", - "resource": { - "resourceType": "Observation", - "id": "d64fc45b-4be9-43ba-b3ef-79ec46006e82", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 78.526, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a8567ef0-11ee-45ec-b6fb-79991775c11c", - "resource": { - "resourceType": "Observation", - "id": "a8567ef0-11ee-45ec-b6fb-79991775c11c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 103, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2da9a643-2834-4d2e-9606-36327310a35f", - "resource": { - "resourceType": "Observation", - "id": "2da9a643-2834-4d2e-9606-36327310a35f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:657a7b45-dfc8-4876-8c6f-3beeceda6948", - "resource": { - "resourceType": "Observation", - "id": "657a7b45-dfc8-4876-8c6f-3beeceda6948", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2be0c9be-5fee-4d62-8534-134722a6c56d", - "resource": { - "resourceType": "Observation", - "id": "2be0c9be-5fee-4d62-8534-134722a6c56d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "effectiveDateTime": "2014-10-30T12:17:06-07:00", - "issued": "2014-10-30T12:17:06.181-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3ef4fa21-4101-4510-b0bc-8971b7b6ac3e", - "resource": { - "resourceType": "Procedure", - "id": "3ef4fa21-4101-4510-b0bc-8971b7b6ac3e", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2014-10-30T12:17:06-07:00", - "end": "2014-10-30T12:32:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:33faacab-5f2f-4c4c-8e4b-5dc2762a6b90", - "resource": { - "resourceType": "MedicationOrder", - "id": "33faacab-5f2f-4c4c-8e4b-5dc2762a6b90", - "dateWritten": "2014-10-30T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:f87bf469-3e34-4ddd-b252-dcc93cc029e4", - "resource": { - "resourceType": "Claim", - "id": "f87bf469-3e34-4ddd-b252-dcc93cc029e4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:33faacab-5f2f-4c4c-8e4b-5dc2762a6b90" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a516b9fd-5cd6-419f-a13a-340c868350b7", - "resource": { - "resourceType": "MedicationOrder", - "id": "a516b9fd-5cd6-419f-a13a-340c868350b7", - "dateWritten": "2014-10-30T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:254a8257-c61d-407d-a7b6-665408cbb5ad", - "resource": { - "resourceType": "Claim", - "id": "254a8257-c61d-407d-a7b6-665408cbb5ad", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a516b9fd-5cd6-419f-a13a-340c868350b7" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:21a17190-cf83-4102-a448-48ac25918efe", - "resource": { - "resourceType": "MedicationOrder", - "id": "21a17190-cf83-4102-a448-48ac25918efe", - "dateWritten": "2014-10-30T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:75307869-f3bb-4c65-bd91-b4379b98b116", - "resource": { - "resourceType": "Claim", - "id": "75307869-f3bb-4c65-bd91-b4379b98b116", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:21a17190-cf83-4102-a448-48ac25918efe" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7a2de518-de8b-400f-98aa-bbdd17c21aa6", - "resource": { - "resourceType": "MedicationOrder", - "id": "7a2de518-de8b-400f-98aa-bbdd17c21aa6", - "dateWritten": "2014-10-30T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:44bb4b83-b03c-44c1-b3c0-bff00599c44d", - "resource": { - "resourceType": "Claim", - "id": "44bb4b83-b03c-44c1-b3c0-bff00599c44d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:7a2de518-de8b-400f-98aa-bbdd17c21aa6" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8cbad22a-0830-4fce-afa8-b3cfbacc5e3f", - "resource": { - "resourceType": "Immunization", - "id": "8cbad22a-0830-4fce-afa8-b3cfbacc5e3f", - "status": "completed", - "date": "2014-10-30T12:17:06-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:6b54f746-9c2e-4a36-a384-59c755778ce4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ca29355e-1b2c-4358-b8fb-8df2b1a9ecfc", - "resource": { - "resourceType": "Claim", - "id": "ca29355e-1b2c-4358-b8fb-8df2b1a9ecfc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 867.25, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383", - "resource": { - "resourceType": "Encounter", - "id": "3071a31a-0bb1-4a7c-828c-1ea626f28383", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2015-11-05T11:17:06-08:00", - "end": "2015-11-05T11:32:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f69a0e97-6390-436a-b6e4-eb3afb4e0f60", - "resource": { - "resourceType": "Observation", - "id": "f69a0e97-6390-436a-b6e4-eb3afb4e0f60", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 104.20, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8c359636-b990-4ffc-8b21-2d8f1733f6c6", - "resource": { - "resourceType": "Observation", - "id": "8c359636-b990-4ffc-8b21-2d8f1733f6c6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a53bb911-3e18-4199-a12b-63efe183f897", - "resource": { - "resourceType": "Observation", - "id": "a53bb911-3e18-4199-a12b-63efe183f897", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 18.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b757eed6-3a53-42af-a532-c4567d864337", - "resource": { - "resourceType": "Observation", - "id": "b757eed6-3a53-42af-a532-c4567d864337", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 17.370, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba466ba4-f1fa-4b4c-ab63-c9d5e5a28770", - "resource": { - "resourceType": "Observation", - "id": "ba466ba4-f1fa-4b4c-ab63-c9d5e5a28770", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 91.180, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:055230ca-681a-466a-a0ed-b208b6ba592b", - "resource": { - "resourceType": "Observation", - "id": "055230ca-681a-466a-a0ed-b208b6ba592b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:044bc207-c84d-4692-b1c2-672701561f5a", - "resource": { - "resourceType": "Observation", - "id": "044bc207-c84d-4692-b1c2-672701561f5a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 66, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf2b2c65-a10d-44dd-9159-80a85e92b111", - "resource": { - "resourceType": "Observation", - "id": "cf2b2c65-a10d-44dd-9159-80a85e92b111", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f1f8911-f9f5-4f10-9ac7-0dc5a7a618b4", - "resource": { - "resourceType": "Observation", - "id": "5f1f8911-f9f5-4f10-9ac7-0dc5a7a618b4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "effectiveDateTime": "2015-11-05T11:17:06-08:00", - "issued": "2015-11-05T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a28093d4-165c-47ba-b68e-60305247601f", - "resource": { - "resourceType": "MedicationOrder", - "id": "a28093d4-165c-47ba-b68e-60305247601f", - "dateWritten": "2015-11-05T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:df327e18-edbc-408c-abc2-2107c9850382", - "resource": { - "resourceType": "Claim", - "id": "df327e18-edbc-408c-abc2-2107c9850382", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a28093d4-165c-47ba-b68e-60305247601f" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3339c556-439b-4e53-9be2-e16a89d6549a", - "resource": { - "resourceType": "MedicationOrder", - "id": "3339c556-439b-4e53-9be2-e16a89d6549a", - "dateWritten": "2015-11-05T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:eb2ac11d-eea3-4469-9154-35facdafa11e", - "resource": { - "resourceType": "Claim", - "id": "eb2ac11d-eea3-4469-9154-35facdafa11e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:3339c556-439b-4e53-9be2-e16a89d6549a" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:915d3f1e-4b07-4733-af36-a761af333ea3", - "resource": { - "resourceType": "MedicationOrder", - "id": "915d3f1e-4b07-4733-af36-a761af333ea3", - "dateWritten": "2015-11-05T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:5d11b35b-8f90-4b98-a36f-d943c965673a", - "resource": { - "resourceType": "Claim", - "id": "5d11b35b-8f90-4b98-a36f-d943c965673a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:915d3f1e-4b07-4733-af36-a761af333ea3" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6a8cd67a-a9df-4d00-9f05-73facb256ff1", - "resource": { - "resourceType": "MedicationOrder", - "id": "6a8cd67a-a9df-4d00-9f05-73facb256ff1", - "dateWritten": "2015-11-05T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:4aab4355-1444-4e35-982f-3fa704301fff", - "resource": { - "resourceType": "Claim", - "id": "4aab4355-1444-4e35-982f-3fa704301fff", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:6a8cd67a-a9df-4d00-9f05-73facb256ff1" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:051bf2d4-3dea-49aa-b51f-5abfc2bc45dc", - "resource": { - "resourceType": "Immunization", - "id": "051bf2d4-3dea-49aa-b51f-5abfc2bc45dc", - "status": "completed", - "date": "2015-11-05T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d8408600-dfb8-46dd-b7ab-1cd897e4cb3a", - "resource": { - "resourceType": "Immunization", - "id": "d8408600-dfb8-46dd-b7ab-1cd897e4cb3a", - "status": "completed", - "date": "2015-11-05T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d2ea0bad-affb-41ef-bbf4-9563fc1ffde7", - "resource": { - "resourceType": "Immunization", - "id": "d2ea0bad-affb-41ef-bbf4-9563fc1ffde7", - "status": "completed", - "date": "2015-11-05T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:808fd435-e586-47e6-8e87-2efa63e6de19", - "resource": { - "resourceType": "Immunization", - "id": "808fd435-e586-47e6-8e87-2efa63e6de19", - "status": "completed", - "date": "2015-11-05T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:88696441-9d6a-432f-8504-d62a81e2548d", - "resource": { - "resourceType": "Immunization", - "id": "88696441-9d6a-432f-8504-d62a81e2548d", - "status": "completed", - "date": "2015-11-05T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3071a31a-0bb1-4a7c-828c-1ea626f28383" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:bcc66b34-3531-41da-8ce4-2ec28a3d21a6", - "resource": { - "resourceType": "Claim", - "id": "bcc66b34-3531-41da-8ce4-2ec28a3d21a6", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3", - "resource": { - "resourceType": "Encounter", - "id": "533c729b-7180-41ef-b222-65a1ad95bad3", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2016-11-10T11:17:06-08:00", - "end": "2016-11-10T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d8adac1d-6f28-4544-9a93-abea9e4f1255", - "resource": { - "resourceType": "Observation", - "id": "d8adac1d-6f28-4544-9a93-abea9e4f1255", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 110.30, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:240db449-ebc7-43f0-b667-43fe68c113b8", - "resource": { - "resourceType": "Observation", - "id": "240db449-ebc7-43f0-b667-43fe68c113b8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cdc17c05-281c-4d65-8b77-f6bf318ccb87", - "resource": { - "resourceType": "Observation", - "id": "cdc17c05-281c-4d65-8b77-f6bf318ccb87", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 22.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2c2837d-34b8-4de8-bc37-66c04123daee", - "resource": { - "resourceType": "Observation", - "id": "e2c2837d-34b8-4de8-bc37-66c04123daee", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 18.740, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:22485690-c64c-4a15-aea8-df046fdcb8cc", - "resource": { - "resourceType": "Observation", - "id": "22485690-c64c-4a15-aea8-df046fdcb8cc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 96.257, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5781b1ab-02a4-416c-993b-3c8da371c4d5", - "resource": { - "resourceType": "Observation", - "id": "5781b1ab-02a4-416c-993b-3c8da371c4d5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c9ae8ca-9b83-4bda-8c58-4bdd43ab18f5", - "resource": { - "resourceType": "Observation", - "id": "9c9ae8ca-9b83-4bda-8c58-4bdd43ab18f5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 87, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25bf47f2-324c-456c-9fce-d40825d9e4db", - "resource": { - "resourceType": "Observation", - "id": "25bf47f2-324c-456c-9fce-d40825d9e4db", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecdfba49-8905-49bf-b774-cc79814b2e96", - "resource": { - "resourceType": "Observation", - "id": "ecdfba49-8905-49bf-b774-cc79814b2e96", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 7.2494, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:575bb625-0686-4cb0-85b2-979d7e4dd3d7", - "resource": { - "resourceType": "Observation", - "id": "575bb625-0686-4cb0-85b2-979d7e4dd3d7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 4.1674, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c248f3a7-3cb3-479b-9229-18caf8495d3d", - "resource": { - "resourceType": "Observation", - "id": "c248f3a7-3cb3-479b-9229-18caf8495d3d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 14.995, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1ce1f94-f907-4605-9c7b-647168c8242b", - "resource": { - "resourceType": "Observation", - "id": "a1ce1f94-f907-4605-9c7b-647168c8242b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 39.933, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:243bf02c-d6f5-4c3c-b5fc-ccba669f6b8e", - "resource": { - "resourceType": "Observation", - "id": "243bf02c-d6f5-4c3c-b5fc-ccba669f6b8e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 80.178, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3cdad9b2-23d2-4fad-93b1-0db950b4b4a0", - "resource": { - "resourceType": "Observation", - "id": "3cdad9b2-23d2-4fad-93b1-0db950b4b4a0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 28.351, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f24d4e5-457f-47de-a4ff-e29dedce4263", - "resource": { - "resourceType": "Observation", - "id": "1f24d4e5-457f-47de-a4ff-e29dedce4263", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 35.565, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa3337bc-9376-4bec-ae1a-6b4f43fd1251", - "resource": { - "resourceType": "Observation", - "id": "fa3337bc-9376-4bec-ae1a-6b4f43fd1251", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 41.654, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6415dfc-9e36-4bb2-87fa-b6b09ec777db", - "resource": { - "resourceType": "Observation", - "id": "e6415dfc-9e36-4bb2-87fa-b6b09ec777db", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 222.68, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d6c05e8-dc0b-4574-b2b6-a5ad00c765e4", - "resource": { - "resourceType": "Observation", - "id": "3d6c05e8-dc0b-4574-b2b6-a5ad00c765e4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 508.47, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:21467e46-8d7d-4191-abcd-75e9fc00fc5a", - "resource": { - "resourceType": "Observation", - "id": "21467e46-8d7d-4191-abcd-75e9fc00fc5a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueQuantity": { - "value": 10.950, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9fe386f9-da6d-4b6a-88c4-cd5fef9ea747", - "resource": { - "resourceType": "Observation", - "id": "9fe386f9-da6d-4b6a-88c4-cd5fef9ea747", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf6d858e-7dd5-4926-8712-bfcd2b27a866", - "resource": { - "resourceType": "Procedure", - "id": "cf6d858e-7dd5-4926-8712-bfcd2b27a866", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2016-11-10T11:17:06-08:00", - "end": "2016-11-10T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c3c48c3f-5e86-4a39-880a-76c153d49af7", - "resource": { - "resourceType": "MedicationOrder", - "id": "c3c48c3f-5e86-4a39-880a-76c153d49af7", - "dateWritten": "2016-11-10T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:ef45b077-97fe-4935-9021-73fd724537a9", - "resource": { - "resourceType": "Claim", - "id": "ef45b077-97fe-4935-9021-73fd724537a9", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:c3c48c3f-5e86-4a39-880a-76c153d49af7" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f33b9e8d-768c-45a1-b990-163ceb96a32b", - "resource": { - "resourceType": "MedicationOrder", - "id": "f33b9e8d-768c-45a1-b990-163ceb96a32b", - "dateWritten": "2016-11-10T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:9be6af68-849b-47dc-9f1f-6249f16a1e86", - "resource": { - "resourceType": "Claim", - "id": "9be6af68-849b-47dc-9f1f-6249f16a1e86", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:f33b9e8d-768c-45a1-b990-163ceb96a32b" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:80aa64ec-931c-4955-b1fd-d9569f059e60", - "resource": { - "resourceType": "MedicationOrder", - "id": "80aa64ec-931c-4955-b1fd-d9569f059e60", - "dateWritten": "2016-11-10T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:f6ae23d6-f5c9-482e-b814-73134aadf7e7", - "resource": { - "resourceType": "Claim", - "id": "f6ae23d6-f5c9-482e-b814-73134aadf7e7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:80aa64ec-931c-4955-b1fd-d9569f059e60" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:27a15798-b2bd-4827-9e9e-49cf65e14d60", - "resource": { - "resourceType": "MedicationOrder", - "id": "27a15798-b2bd-4827-9e9e-49cf65e14d60", - "dateWritten": "2016-11-10T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:eb5af8f9-9901-43e5-8aec-ce04ffebc6db", - "resource": { - "resourceType": "Claim", - "id": "eb5af8f9-9901-43e5-8aec-ce04ffebc6db", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:27a15798-b2bd-4827-9e9e-49cf65e14d60" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:91cd35b7-c51a-4fe3-a5c5-7190e09f2db1", - "resource": { - "resourceType": "Immunization", - "id": "91cd35b7-c51a-4fe3-a5c5-7190e09f2db1", - "status": "completed", - "date": "2016-11-10T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8cca8ef6-54a3-44e6-9502-aa55f640e173", - "resource": { - "resourceType": "DiagnosticReport", - "id": "8cca8ef6-54a3-44e6-9502-aa55f640e173", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:533c729b-7180-41ef-b222-65a1ad95bad3" - }, - "effectiveDateTime": "2016-11-10T11:17:06-08:00", - "issued": "2016-11-10T11:17:06.181-08:00", - "performer": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "result": [ - { - "reference": "urn:uuid:21467e46-8d7d-4191-abcd-75e9fc00fc5a", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:fe2db6c9-0f90-4a67-8bca-d1e856aecbfd", - "resource": { - "resourceType": "Claim", - "id": "fe2db6c9-0f90-4a67-8bca-d1e856aecbfd", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 557.19, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4", - "resource": { - "resourceType": "Encounter", - "id": "6c2fb27d-edac-4580-a465-21ec775ac4b4", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2017-11-16T11:17:06-08:00", - "end": "2017-11-16T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5a110c03-b655-4048-8b41-30410dd30b2e", - "resource": { - "resourceType": "Observation", - "id": "5a110c03-b655-4048-8b41-30410dd30b2e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 116.40, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e62a2a09-f6f2-4c74-8ae0-1e7c6b907b20", - "resource": { - "resourceType": "Observation", - "id": "e62a2a09-f6f2-4c74-8ae0-1e7c6b907b20", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28163514-80d7-4a03-953a-24ecd6d0c7a1", - "resource": { - "resourceType": "Observation", - "id": "28163514-80d7-4a03-953a-24ecd6d0c7a1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 24.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:199be083-0820-4530-98e7-30520b484a49", - "resource": { - "resourceType": "Observation", - "id": "199be083-0820-4530-98e7-30520b484a49", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 18.400, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d96a45d-31f9-4052-9211-02897e01f173", - "resource": { - "resourceType": "Observation", - "id": "2d96a45d-31f9-4052-9211-02897e01f173", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 92.406, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b118da3-b917-4a4c-a4b1-ed7fcc5e0e86", - "resource": { - "resourceType": "Observation", - "id": "3b118da3-b917-4a4c-a4b1-ed7fcc5e0e86", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65e6e6f0-381c-4a77-a112-a4b8316bbce7", - "resource": { - "resourceType": "Observation", - "id": "65e6e6f0-381c-4a77-a112-a4b8316bbce7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 71, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:742e140e-6652-4dc6-bddc-59d2b9344500", - "resource": { - "resourceType": "Observation", - "id": "742e140e-6652-4dc6-bddc-59d2b9344500", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8cb5fce5-b3c5-461c-8d1b-2b30f17ffc72", - "resource": { - "resourceType": "Observation", - "id": "8cb5fce5-b3c5-461c-8d1b-2b30f17ffc72", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "effectiveDateTime": "2017-11-16T11:17:06-08:00", - "issued": "2017-11-16T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4222c16-78e9-409e-a511-17015a5312e9", - "resource": { - "resourceType": "Procedure", - "id": "c4222c16-78e9-409e-a511-17015a5312e9", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2017-11-16T11:17:06-08:00", - "end": "2017-11-16T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e34d43e2-4293-4c08-9352-ecacc35e4dc0", - "resource": { - "resourceType": "MedicationOrder", - "id": "e34d43e2-4293-4c08-9352-ecacc35e4dc0", - "dateWritten": "2017-11-16T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:f60394ca-d448-41ed-b802-6bf44e2c0b96", - "resource": { - "resourceType": "Claim", - "id": "f60394ca-d448-41ed-b802-6bf44e2c0b96", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:e34d43e2-4293-4c08-9352-ecacc35e4dc0" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4354e8e4-90bd-40aa-9b19-a2ff2f67be6b", - "resource": { - "resourceType": "MedicationOrder", - "id": "4354e8e4-90bd-40aa-9b19-a2ff2f67be6b", - "dateWritten": "2017-11-16T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:2ddaf801-4bcd-4c24-9ba9-5debc0b7ad8a", - "resource": { - "resourceType": "Claim", - "id": "2ddaf801-4bcd-4c24-9ba9-5debc0b7ad8a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:4354e8e4-90bd-40aa-9b19-a2ff2f67be6b" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dd249c62-56ec-4bec-8a33-7fbeea9bab14", - "resource": { - "resourceType": "MedicationOrder", - "id": "dd249c62-56ec-4bec-8a33-7fbeea9bab14", - "dateWritten": "2017-11-16T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:830198ff-b1d2-46e4-a6d7-de73909818c8", - "resource": { - "resourceType": "Claim", - "id": "830198ff-b1d2-46e4-a6d7-de73909818c8", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:dd249c62-56ec-4bec-8a33-7fbeea9bab14" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:79641136-2ad7-4d00-9de7-32294cd74eda", - "resource": { - "resourceType": "MedicationOrder", - "id": "79641136-2ad7-4d00-9de7-32294cd74eda", - "dateWritten": "2017-11-16T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:aa6ba46f-c48c-4eec-9fdf-07159f42f421", - "resource": { - "resourceType": "Claim", - "id": "aa6ba46f-c48c-4eec-9fdf-07159f42f421", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:79641136-2ad7-4d00-9de7-32294cd74eda" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:35b046a9-4986-4781-b7fd-88abb7a2c9cd", - "resource": { - "resourceType": "Immunization", - "id": "35b046a9-4986-4781-b7fd-88abb7a2c9cd", - "status": "completed", - "date": "2017-11-16T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:6c2fb27d-edac-4580-a465-21ec775ac4b4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:57538e1f-c700-420b-bcdd-b60f4feaf110", - "resource": { - "resourceType": "Claim", - "id": "57538e1f-c700-420b-bcdd-b60f4feaf110", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 545.88, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5", - "resource": { - "resourceType": "Encounter", - "id": "2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2018-11-22T11:17:06-08:00", - "end": "2018-11-22T11:32:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4f7d9ad8-9cb2-4330-b5be-6227cafed4f9", - "resource": { - "resourceType": "Observation", - "id": "4f7d9ad8-9cb2-4330-b5be-6227cafed4f9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 122.20, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:398a2817-acf4-4e06-8a9b-80b8cb14fd4c", - "resource": { - "resourceType": "Observation", - "id": "398a2817-acf4-4e06-8a9b-80b8cb14fd4c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9e0cca73-0ead-4cb9-8082-75cd10348096", - "resource": { - "resourceType": "Observation", - "id": "9e0cca73-0ead-4cb9-8082-75cd10348096", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 29.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0b32780-90b6-4a96-b1d4-4c0e4df31079", - "resource": { - "resourceType": "Observation", - "id": "c0b32780-90b6-4a96-b1d4-4c0e4df31079", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 19.700, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb50cbed-1da2-418d-82b8-103c5879b300", - "resource": { - "resourceType": "Observation", - "id": "fb50cbed-1da2-418d-82b8-103c5879b300", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 94.268, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f63355a7-7e03-47c2-a51b-0716be99660f", - "resource": { - "resourceType": "Observation", - "id": "f63355a7-7e03-47c2-a51b-0716be99660f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 139, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3fe321ad-2e98-4ec4-8f36-6b0329332191", - "resource": { - "resourceType": "Observation", - "id": "3fe321ad-2e98-4ec4-8f36-6b0329332191", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 66, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d3a4e85-0439-4a99-87e9-9c09836a1849", - "resource": { - "resourceType": "Observation", - "id": "7d3a4e85-0439-4a99-87e9-9c09836a1849", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b034145c-ed4d-47fc-a767-a233573a7279", - "resource": { - "resourceType": "Observation", - "id": "b034145c-ed4d-47fc-a767-a233573a7279", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "effectiveDateTime": "2018-11-22T11:17:06-08:00", - "issued": "2018-11-22T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e37c807-27e4-4dba-871f-5ac81870dff7", - "resource": { - "resourceType": "MedicationOrder", - "id": "7e37c807-27e4-4dba-871f-5ac81870dff7", - "dateWritten": "2018-11-22T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:21752dfc-cc7f-4d9b-8c88-3641b3e02310", - "resource": { - "resourceType": "Claim", - "id": "21752dfc-cc7f-4d9b-8c88-3641b3e02310", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:7e37c807-27e4-4dba-871f-5ac81870dff7" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a8c49bde-9362-4d88-9861-082583d639de", - "resource": { - "resourceType": "MedicationOrder", - "id": "a8c49bde-9362-4d88-9861-082583d639de", - "dateWritten": "2018-11-22T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:44fdb5f3-28f9-4cd2-9ac9-9ae9ada442f3", - "resource": { - "resourceType": "Claim", - "id": "44fdb5f3-28f9-4cd2-9ac9-9ae9ada442f3", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:a8c49bde-9362-4d88-9861-082583d639de" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:426e17e4-7eea-4cf5-9a6c-b37ef790b4e4", - "resource": { - "resourceType": "MedicationOrder", - "id": "426e17e4-7eea-4cf5-9a6c-b37ef790b4e4", - "dateWritten": "2018-11-22T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:defc2cb2-3ba0-40be-95ac-d78d7aa9faaa", - "resource": { - "resourceType": "Claim", - "id": "defc2cb2-3ba0-40be-95ac-d78d7aa9faaa", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:426e17e4-7eea-4cf5-9a6c-b37ef790b4e4" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b4e92f90-4457-4696-a774-ff4d38c07ca4", - "resource": { - "resourceType": "MedicationOrder", - "id": "b4e92f90-4457-4696-a774-ff4d38c07ca4", - "dateWritten": "2018-11-22T11:17:06-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:659506dd-ee0f-4708-9336-da59f570ad4a", - "resource": { - "resourceType": "Claim", - "id": "659506dd-ee0f-4708-9336-da59f570ad4a", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:b4e92f90-4457-4696-a774-ff4d38c07ca4" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:36fd48c5-a86f-462f-a797-ff90790076d6", - "resource": { - "resourceType": "Immunization", - "id": "36fd48c5-a86f-462f-a797-ff90790076d6", - "status": "completed", - "date": "2018-11-22T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:2d5d87e4-8396-41fc-b8f7-bee29f6d6fc5" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2ec6ecb9-1817-45c5-9cae-9d2fbad392c2", - "resource": { - "resourceType": "Claim", - "id": "2ec6ecb9-1817-45c5-9cae-9d2fbad392c2", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c", - "resource": { - "resourceType": "Organization", - "id": "b7175ab4-bde5-3848-891b-579bccb77c7c", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "b7175ab4-bde5-3848-891b-579bccb77c7c" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "TUFTS MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "6176365000" - } - ], - "address": [ - { - "line": [ - "800 WASHINGTON STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02111", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044", - "resource": { - "resourceType": "Practitioner", - "id": "002862dc-5ff1-380e-82ad-a3cf9c436044", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "470" - } - ], - "active": true, - "name": { - "family": [ - "Orn563" - ], - "given": [ - "Serina556" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "800 WASHINGTON STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02111", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a", - "resource": { - "resourceType": "Encounter", - "id": "2d0f382b-c1c9-477e-ad42-0cdae1d1db4a", - "status": "finished", - "class": "emergency", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044" - } - } - ], - "period": { - "start": "2019-08-10T12:17:06-07:00", - "end": "2019-08-10T14:47:06-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b", - "resource": { - "resourceType": "Condition", - "id": "d8c9c5ad-1dfc-4999-bfe1-77c931570f2b", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" - }, - "dateRecorded": "2019-08-10", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58150001", - "display": "Fracture of clavicle" - } - ], - "text": "Fracture of clavicle" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2019-08-10T12:17:06-07:00", - "abatementDateTime": "2019-10-09T12:17:06-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:6d659beb-5b35-472f-97bb-e1ed3f0bdd8c", - "resource": { - "resourceType": "Procedure", - "id": "6d659beb-5b35-472f-97bb-e1ed3f0bdd8c", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "168594001", - "display": "Clavicle X-ray" - } - ], - "text": "Clavicle X-ray" - }, - "performedPeriod": { - "start": "2019-08-10T12:17:06-07:00", - "end": "2019-08-10T12:47:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:aee2aef3-09ef-4bfc-b352-0d7123e2af30", - "resource": { - "resourceType": "Procedure", - "id": "aee2aef3-09ef-4bfc-b352-0d7123e2af30", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "305428000", - "display": "Admission to orthopedic department" - } - ], - "text": "Admission to orthopedic department" - }, - "reasonReference": { - "reference": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b" - }, - "performedPeriod": { - "start": "2019-08-10T12:17:06-07:00", - "end": "2019-08-10T13:17:06-07:00" - }, - "encounter": { - "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:92365c53-0cdd-4cc5-b35b-bebb279de3ee", - "resource": { - "resourceType": "MedicationOrder", - "id": "92365c53-0cdd-4cc5-b35b-bebb279de3ee", - "dateWritten": "2019-08-10T12:17:06-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044" - }, - "encounter": { - "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:8fd60b70-ee51-4c3c-ac2b-d93c419e8e18", - "resource": { - "resourceType": "Claim", - "id": "8fd60b70-ee51-4c3c-ac2b-d93c419e8e18", - "type": "institutional", - "organization": { - "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:92365c53-0cdd-4cc5-b35b-bebb279de3ee" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9a4863d1-15d6-4779-964a-5432113a3773", - "resource": { - "resourceType": "CarePlan", - "id": "9a4863d1-15d6-4779-964a-5432113a3773", - "text": { - "status": "generated", - "div": "
    Fracture care
    " - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "context": { - "reference": "urn:uuid:2d0f382b-c1c9-477e-ad42-0cdae1d1db4a" - }, - "period": { - "start": "2019-08-10T12:17:06-07:00", - "end": "2019-10-09T12:17:06-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385691007", - "display": "Fracture care" - } - ], - "text": "Fracture care" - } - ], - "addresses": [ - { - "reference": "urn:uuid:d8c9c5ad-1dfc-4999-bfe1-77c931570f2b" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "183051005", - "display": "Recommendation to rest" - } - ], - "text": "Recommendation to rest" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "408580007", - "display": "Physical activity target light exercise" - } - ], - "text": "Physical activity target light exercise" - }, - "status": "completed", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:c8157528-c44b-4964-8b9b-8e5d8d546b87", - "resource": { - "resourceType": "ImagingStudy", - "id": "c8157528-c44b-4964-8b9b-8e5d8d546b87", - "started": "2019-08-10T12:17:06-07:00", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "uid": "urn:oid:1.2.840.99999999.45493348.1586309773119", - "numberOfSeries": 1, - "numberOfInstances": 1, - "series": [ - { - "number": 1, - "modality": { - "system": "http://dicom.nema.org/resources/ontology/DCM", - "code": "DX", - "display": "Digital Radiography" - }, - "uid": "urn:oid:1.2.840.99999999.1.49310789.1586309773119", - "numberOfInstances": 1, - "availability": "UNAVAILABLE", - "bodySite": { - "system": "http://snomed.info/sct", - "code": "51299004", - "display": "Clavicle" - }, - "started": "2019-08-10T12:17:06-07:00", - "instance": [ - { - "number": 1, - "uid": "urn:oid:1.2.840.99999999.1.1.61194548.1586309773119", - "sopClass": "urn:oid:1.2.840.10008.5.1.4.1.1.1.1", - "title": "Image of clavicle" - } - ] - } - ] - }, - "request": { - "method": "POST", - "url": "ImagingStudy" - } - }, - { - "fullUrl": "urn:uuid:328f5fab-d3ff-4aa9-8be6-ea12fefd5095", - "resource": { - "resourceType": "Claim", - "id": "328f5fab-d3ff-4aa9-8be6-ea12fefd5095", - "type": "institutional", - "organization": { - "reference": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "58150001", - "display": "Fracture of clavicle" - } - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "168594001", - "display": "Clavicle X-ray" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "305428000", - "display": "Admission to orthopedic department" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3", - "resource": { - "resourceType": "Organization", - "id": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "BOSTON MEDICAL CENTER CORPORATION-", - "telecom": [ - { - "system": "phone", - "value": "6176388000" - } - ], - "address": [ - { - "line": [ - "1 BOSTON MEDICAL CENTER PLACE" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02118", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", - "resource": { - "resourceType": "Practitioner", - "id": "64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "140" - } - ], - "active": true, - "name": { - "family": [ - "Kuvalis369" - ], - "given": [ - "Maricruz991" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "1 BOSTON MEDICAL CENTER PLACE" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02118", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:087434d3-9ce1-479e-a0f3-c5f6db5ba171", - "resource": { - "resourceType": "Encounter", - "id": "087434d3-9ce1-479e-a0f3-c5f6db5ba171", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd" - } - } - ], - "period": { - "start": "2019-10-09T12:17:06-07:00", - "end": "2019-10-09T12:32:06-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58150001", - "display": "Fracture of clavicle" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:02e30739-43b7-4a95-b94a-3e84910367fa", - "resource": { - "resourceType": "Claim", - "id": "02e30739-43b7-4a95-b94a-3e84910367fa", - "type": "institutional", - "organization": { - "reference": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce", - "resource": { - "resourceType": "Encounter", - "id": "671ab4ad-1066-4bde-b53e-492724b2f7ce", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - } - } - ], - "period": { - "start": "2019-11-28T11:17:06-08:00", - "end": "2019-11-28T11:47:06-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:953671bc-f307-46b4-8245-5320edf44237", - "resource": { - "resourceType": "Observation", - "id": "953671bc-f307-46b4-8245-5320edf44237", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 127.90, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c3f1068-32e6-41ab-9c2e-2beea381d130", - "resource": { - "resourceType": "Observation", - "id": "9c3f1068-32e6-41ab-9c2e-2beea381d130", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7a1fce1a-41a2-4eda-b38f-7fb2ad67fed9", - "resource": { - "resourceType": "Observation", - "id": "7a1fce1a-41a2-4eda-b38f-7fb2ad67fed9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 34.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:385d3583-6f5d-4456-9fd4-403ad976d46d", - "resource": { - "resourceType": "Observation", - "id": "385d3583-6f5d-4456-9fd4-403ad976d46d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 20.980, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2eb36260-32fa-42cf-a0d9-1bb621f6fd77", - "resource": { - "resourceType": "Observation", - "id": "2eb36260-32fa-42cf-a0d9-1bb621f6fd77", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 94.779, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fec7f1f1-0376-4397-aeb8-259de0c0bec3", - "resource": { - "resourceType": "Observation", - "id": "fec7f1f1-0376-4397-aeb8-259de0c0bec3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 109, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:adbe6725-fd67-499e-b7ff-c5d74b17e658", - "resource": { - "resourceType": "Observation", - "id": "adbe6725-fd67-499e-b7ff-c5d74b17e658", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 60, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2034e61f-7ee0-4543-9616-45e80ae7a7c9", - "resource": { - "resourceType": "Observation", - "id": "2034e61f-7ee0-4543-9616-45e80ae7a7c9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0e4c867-026d-4147-ab03-915d7bb49730", - "resource": { - "resourceType": "Observation", - "id": "b0e4c867-026d-4147-ab03-915d7bb49730", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "effectiveDateTime": "2019-11-28T11:17:06-08:00", - "issued": "2019-11-28T11:17:06.181-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f48d78f9-fa02-45e5-ba14-8a608a2f4d73", - "resource": { - "resourceType": "Procedure", - "id": "f48d78f9-fa02-45e5-ba14-8a608a2f4d73", - "subject": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2019-11-28T11:17:06-08:00", - "end": "2019-11-28T11:32:06-08:00" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5871c437-3957-4f84-aa19-0818bd4d52ae", - "resource": { - "resourceType": "MedicationOrder", - "id": "5871c437-3957-4f84-aa19-0818bd4d52ae", - "dateWritten": "2019-11-28T11:17:06-08:00", - "status": "active", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309362", - "display": "Clopidogrel 75 MG Oral Tablet" - } - ], - "text": "Clopidogrel 75 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:3d2e5927-6707-442a-a1cc-fe42c7ba7c81", - "resource": { - "resourceType": "Claim", - "id": "3d2e5927-6707-442a-a1cc-fe42c7ba7c81", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:5871c437-3957-4f84-aa19-0818bd4d52ae" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:49f918c7-1033-44da-9599-993bef505b71", - "resource": { - "resourceType": "MedicationOrder", - "id": "49f918c7-1033-44da-9599-993bef505b71", - "dateWritten": "2019-11-28T11:17:06-08:00", - "status": "active", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "705129", - "display": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - ], - "text": "Nitroglycerin 0.4 MG/ACTUAT Mucosal Spray" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:8467dda5-39a4-4b18-a907-5bfdf3891b80", - "resource": { - "resourceType": "Claim", - "id": "8467dda5-39a4-4b18-a907-5bfdf3891b80", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:49f918c7-1033-44da-9599-993bef505b71" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2fbd2f03-35d4-48c9-8313-be8bebbc2e5a", - "resource": { - "resourceType": "MedicationOrder", - "id": "2fbd2f03-35d4-48c9-8313-be8bebbc2e5a", - "dateWritten": "2019-11-28T11:17:06-08:00", - "status": "active", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312961", - "display": "Simvastatin 20 MG Oral Tablet" - } - ], - "text": "Simvastatin 20 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:97313763-78ef-43b2-9055-5d3eddda39e3", - "resource": { - "resourceType": "Claim", - "id": "97313763-78ef-43b2-9055-5d3eddda39e3", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:2fbd2f03-35d4-48c9-8313-be8bebbc2e5a" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4ccb9509-e4c5-4514-b36d-d239b93fdace", - "resource": { - "resourceType": "MedicationOrder", - "id": "4ccb9509-e4c5-4514-b36d-d239b93fdace", - "dateWritten": "2019-11-28T11:17:06-08:00", - "status": "active", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "prescriber": { - "reference": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e" - }, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "197361", - "display": "Amlodipine 5 MG Oral Tablet" - } - ], - "text": "Amlodipine 5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:ffdd6a17-fab2-4402-a7f9-7e306ef1e9b3", - "resource": { - "resourceType": "Claim", - "id": "ffdd6a17-fab2-4402-a7f9-7e306ef1e9b3", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:4ccb9509-e4c5-4514-b36d-d239b93fdace" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9dba0685-9a01-4bd1-b3a2-6f75960af8c1", - "resource": { - "resourceType": "Immunization", - "id": "9dba0685-9a01-4bd1-b3a2-6f75960af8c1", - "status": "completed", - "date": "2019-11-28T11:17:06-08:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:671ab4ad-1066-4bde-b53e-492724b2f7ce" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d8a265a5-6188-4f85-8828-7fb12be9995b", - "resource": { - "resourceType": "Claim", - "id": "d8a265a5-6188-4f85-8828-7fb12be9995b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:4ecb4cbb-6df7-41e0-8e89-6e7a142721a5" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 473.18, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json deleted file mode 100644 index 67e63f747028..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Kortney212_Bosco882_8ed31d3e-7352-4055-89c4-f017db3f594e.json +++ /dev/null @@ -1,9279 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e", - "resource": { - "resourceType": "Patient", - "id": "8ed31d3e-7352-4055-89c4-f017db3f594e", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -1966016355271311648 Population seed: 1586309754086
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Race", - "code": "2106-3", - "display": "White" - } - ], - "text": "White" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Ethnicity", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - ], - "text": "Not Hispanic or Latino" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Reda120 Franecki195" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Wrentham", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.9331359488509279 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 57.066864051149075 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR" - } - ] - }, - "system": "http://hospital.smarthealthit.org", - "value": "8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB" - } - ] - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-87-6568" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "DL" - } - ] - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99945489" - } - ], - "name": [ - { - "use": "official", - "family": [ - "Bosco882" - ], - "given": [ - "Kortney212" - ], - "prefix": [ - "Ms." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-409-5117", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1961-06-28", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.422646574026295 - }, - { - "url": "longitude", - "valueDecimal": -71.18816183585602 - } - ] - } - ], - "line": [ - "475 Parker Rapid" - ], - "city": "Winchester", - "state": "Massachusetts", - "postalCode": "01890", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S" - } - ] - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219", - "resource": { - "resourceType": "Organization", - "id": "245f252c-be87-3017-8a0b-a04448a97219", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "245f252c-be87-3017-8a0b-a04448a97219" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "PCP45531", - "telecom": [ - { - "system": "phone", - "value": "617-230-9940" - } - ], - "address": [ - { - "line": [ - "661 MASSACHUSETTS AVE" - ], - "city": "ARLINGTON", - "state": "MA", - "postalCode": "02476-5001", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240", - "resource": { - "resourceType": "Practitioner", - "id": "8e9a1427-af53-3468-9d67-db3b9191c240", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "28020" - } - ], - "active": true, - "name": { - "family": [ - "Waters156" - ], - "given": [ - "Damien170" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "661 MASSACHUSETTS AVE" - ], - "city": "ARLINGTON", - "state": "MA", - "postalCode": "02476-5001", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:4512de23-4f75-4dd9-8ed0-48dd04d7e883", - "resource": { - "resourceType": "Encounter", - "id": "4512de23-4f75-4dd9-8ed0-48dd04d7e883", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "1989-09-06T12:00:58-07:00", - "end": "1989-09-06T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ea898089-97dc-4f22-a259-8d8d1d4dd5a1", - "resource": { - "resourceType": "Condition", - "id": "ea898089-97dc-4f22-a259-8d8d1d4dd5a1", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4512de23-4f75-4dd9-8ed0-48dd04d7e883" - }, - "dateRecorded": "1989-09-06", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "1989-09-06T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:41d47904-32ae-4b13-8ac9-ec8ca0bfb521", - "resource": { - "resourceType": "Claim", - "id": "41d47904-32ae-4b13-8ac9-ec8ca0bfb521", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7", - "resource": { - "resourceType": "Organization", - "id": "f4e7709c-02f6-37ca-aeea-8247d74e88e7", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "WINCHESTER HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "7817299000" - } - ], - "address": [ - { - "line": [ - "41 HIGHLAND AVENUE" - ], - "city": "WINCHESTER", - "state": "MA", - "postalCode": "01890", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203", - "resource": { - "resourceType": "Practitioner", - "id": "a0153fac-7137-30d8-bb5a-0cb7af968203", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "430" - } - ], - "active": true, - "name": { - "family": [ - "Carroll471" - ], - "given": [ - "Emmitt44" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "41 HIGHLAND AVENUE" - ], - "city": "WINCHESTER", - "state": "MA", - "postalCode": "01890", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290", - "resource": { - "resourceType": "Encounter", - "id": "99e80e98-cc07-431a-855e-983e509b9290", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2010-05-19T12:00:58-07:00", - "end": "2010-05-19T12:45:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee", - "resource": { - "resourceType": "Condition", - "id": "d1813c80-97de-4641-b3d9-e0ddede862ee", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" - }, - "dateRecorded": "2010-05-19", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ], - "text": "Normal pregnancy" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-05-19T12:00:58-07:00", - "abatementDateTime": "2010-05-26T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:c0be40f7-2570-4dfb-99a4-e151160140a3", - "resource": { - "resourceType": "Condition", - "id": "c0be40f7-2570-4dfb-99a4-e151160140a3", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" - }, - "dateRecorded": "2010-05-19", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - ], - "text": "Miscarriage in first trimester" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-05-19T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:7200df62-2767-4df5-8209-cd1c83527c78", - "resource": { - "resourceType": "Condition", - "id": "7200df62-2767-4df5-8209-cd1c83527c78", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" - }, - "dateRecorded": "2010-05-19", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "156073000", - "display": "Fetus with unknown complication" - } - ], - "text": "Fetus with unknown complication" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-05-19T12:00:58-07:00", - "abatementDateTime": "2010-05-26T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f6960f74-9a0c-4f10-9da6-b01282cdb84e", - "resource": { - "resourceType": "Procedure", - "id": "f6960f74-9a0c-4f10-9da6-b01282cdb84e", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "252160004", - "display": "Standard pregnancy test" - } - ], - "text": "Standard pregnancy test" - }, - "reasonReference": { - "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" - }, - "performedPeriod": { - "start": "2010-05-19T12:00:58-07:00", - "end": "2010-05-19T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:505b8a61-94ef-4a73-a8c3-c5212c97026d", - "resource": { - "resourceType": "Procedure", - "id": "505b8a61-94ef-4a73-a8c3-c5212c97026d", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "169230002", - "display": "Ultrasound scan for fetal viability" - } - ], - "text": "Ultrasound scan for fetal viability" - }, - "reasonReference": { - "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" - }, - "performedPeriod": { - "start": "2010-05-19T12:00:58-07:00", - "end": "2010-05-19T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:99e80e98-cc07-431a-855e-983e509b9290" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:cf50630e-4990-4ef1-b265-fd28fa193833", - "resource": { - "resourceType": "Claim", - "id": "cf50630e-4990-4ef1-b265-fd28fa193833", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - }, - { - "sequence": 2, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - }, - { - "sequence": 3, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "156073000", - "display": "Fetus with unknown complication" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "252160004", - "display": "Standard pregnancy test" - }, - "net": { - "value": 7287.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "169230002", - "display": "Ultrasound scan for fetal viability" - }, - "net": { - "value": 11149.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3", - "resource": { - "resourceType": "Encounter", - "id": "6dbc35fe-335f-41e3-8e8e-03472aacd1c3", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2010-05-26T12:00:58-07:00", - "end": "2010-05-26T12:45:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "156073000", - "display": "Fetus with unknown complication" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9158ff31-11f1-47af-a810-2d6bcbed613b", - "resource": { - "resourceType": "Procedure", - "id": "9158ff31-11f1-47af-a810-2d6bcbed613b", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination" - } - ], - "text": "Physical examination" - }, - "reasonReference": { - "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" - }, - "performedPeriod": { - "start": "2010-05-26T12:00:58-07:00", - "end": "2010-05-26T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2a5c8470-f88f-4a9a-864f-41896b228931", - "resource": { - "resourceType": "Procedure", - "id": "2a5c8470-f88f-4a9a-864f-41896b228931", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "171207006", - "display": "Depression screening" - } - ], - "text": "Depression screening" - }, - "reasonReference": { - "reference": "urn:uuid:d1813c80-97de-4641-b3d9-e0ddede862ee" - }, - "performedPeriod": { - "start": "2010-05-26T12:00:58-07:00", - "end": "2010-05-26T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:6dbc35fe-335f-41e3-8e8e-03472aacd1c3" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a9cc55fa-d313-466c-a27f-da7e9fb418c2", - "resource": { - "resourceType": "Claim", - "id": "a9cc55fa-d313-466c-a27f-da7e9fb418c2", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical examination" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "171207006", - "display": "Depression screening" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e96e887a-f63c-4d40-a800-2e529fb023b2", - "resource": { - "resourceType": "Encounter", - "id": "e96e887a-f63c-4d40-a800-2e529fb023b2", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2010-08-24T12:00:58-07:00", - "end": "2010-08-24T12:15:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d9faa764-02da-4f93-b161-e93fa8ddde6a", - "resource": { - "resourceType": "Condition", - "id": "d9faa764-02da-4f93-b161-e93fa8ddde6a", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:e96e887a-f63c-4d40-a800-2e529fb023b2" - }, - "dateRecorded": "2010-08-24", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-08-24T12:00:58-07:00", - "abatementDateTime": "2010-09-07T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:1ef2e389-18c3-406b-9538-7c0ea93a1a33", - "resource": { - "resourceType": "Claim", - "id": "1ef2e389-18c3-406b-9538-7c0ea93a1a33", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190", - "resource": { - "resourceType": "Encounter", - "id": "5e5de478-9759-429a-a50f-0d3960846190", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2011-06-29T12:00:58-07:00", - "end": "2011-06-29T12:15:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5d79d370-13a2-42e3-b1ec-c4e040642bb4", - "resource": { - "resourceType": "Observation", - "id": "5d79d370-13a2-42e3-b1ec-c4e040642bb4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:72746c63-ce6a-46f8-8e6a-7f6207fa433f", - "resource": { - "resourceType": "Observation", - "id": "72746c63-ce6a-46f8-8e6a-7f6207fa433f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c2e5d37-8ce6-4b1e-9b38-45f08a84bdb7", - "resource": { - "resourceType": "Observation", - "id": "4c2e5d37-8ce6-4b1e-9b38-45f08a84bdb7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:04c4da0c-07ae-4fd6-b803-a0b6cdfdf3e8", - "resource": { - "resourceType": "Observation", - "id": "04c4da0c-07ae-4fd6-b803-a0b6cdfdf3e8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf32795b-0510-43df-94e4-34036b0bbdd6", - "resource": { - "resourceType": "Observation", - "id": "cf32795b-0510-43df-94e4-34036b0bbdd6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 100, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a0cd6ca8-264f-43f2-a680-d381db876a71", - "resource": { - "resourceType": "Observation", - "id": "a0cd6ca8-264f-43f2-a680-d381db876a71", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c1771b6-a801-4dff-a054-9812dbdef71a", - "resource": { - "resourceType": "Observation", - "id": "9c1771b6-a801-4dff-a054-9812dbdef71a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6c66c74-2984-48f1-bed8-518e6f3e4878", - "resource": { - "resourceType": "Observation", - "id": "c6c66c74-2984-48f1-bed8-518e6f3e4878", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 194.99, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e0f29a6b-61ce-4b86-a741-2dc5046f7d83", - "resource": { - "resourceType": "Observation", - "id": "e0f29a6b-61ce-4b86-a741-2dc5046f7d83", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 104.79, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d241802-e659-4374-a695-bed1bbfb00c8", - "resource": { - "resourceType": "Observation", - "id": "0d241802-e659-4374-a695-bed1bbfb00c8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 110.85, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4e0f7d3-cef0-4b79-bb66-de7a9605794d", - "resource": { - "resourceType": "Observation", - "id": "c4e0f7d3-cef0-4b79-bb66-de7a9605794d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueQuantity": { - "value": 63.190, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2366fedc-da48-4d67-a8ec-67f4dd6f5f36", - "resource": { - "resourceType": "Observation", - "id": "2366fedc-da48-4d67-a8ec-67f4dd6f5f36", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac7e2e9b-0447-4a6e-8724-d3fc639f4a1e", - "resource": { - "resourceType": "Immunization", - "id": "ac7e2e9b-0447-4a6e-8724-d3fc639f4a1e", - "status": "completed", - "date": "2011-06-29T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f9075d2d-6a41-4b82-a158-76f5796a7ca0", - "resource": { - "resourceType": "Immunization", - "id": "f9075d2d-6a41-4b82-a158-76f5796a7ca0", - "status": "completed", - "date": "2011-06-29T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b5f5e6f7-0d4c-464a-9b2f-b6a7197cfb18", - "resource": { - "resourceType": "DiagnosticReport", - "id": "b5f5e6f7-0d4c-464a-9b2f-b6a7197cfb18", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:5e5de478-9759-429a-a50f-0d3960846190" - }, - "effectiveDateTime": "2011-06-29T12:00:58-07:00", - "issued": "2011-06-29T12:00:58.308-07:00", - "performer": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "result": [ - { - "reference": "urn:uuid:c4e0f7d3-cef0-4b79-bb66-de7a9605794d", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:dbe7f641-9b9a-4f44-9355-aa1b8586d56e", - "resource": { - "resourceType": "Claim", - "id": "dbe7f641-9b9a-4f44-9355-aa1b8586d56e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9ee61fc6-9fe8-4616-a7bb-683ee034da6c", - "resource": { - "resourceType": "Encounter", - "id": "9ee61fc6-9fe8-4616-a7bb-683ee034da6c", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2011-06-29T12:00:58-07:00", - "end": "2011-06-29T12:53:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e55ecd48-77a1-4c51-aab1-14838de89810", - "resource": { - "resourceType": "Procedure", - "id": "e55ecd48-77a1-4c51-aab1-14838de89810", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "performedPeriod": { - "start": "2011-06-29T12:00:58-07:00", - "end": "2011-06-29T12:38:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:9ee61fc6-9fe8-4616-a7bb-683ee034da6c" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b268cef1-5fd9-4385-8223-75eb77400f6e", - "resource": { - "resourceType": "Claim", - "id": "b268cef1-5fd9-4385-8223-75eb77400f6e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - }, - "net": { - "value": 12853.05, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4", - "resource": { - "resourceType": "Encounter", - "id": "1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2012-06-15T12:00:58-07:00", - "end": "2012-06-15T12:15:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "201834006", - "display": "Localized, primary osteoarthritis of the hand" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68", - "resource": { - "resourceType": "Condition", - "id": "1dd1524c-d3bd-47d6-b99e-f51d62988a68", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" - }, - "dateRecorded": "2012-06-15", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "201834006", - "display": "Localized, primary osteoarthritis of the hand" - } - ], - "text": "Localized, primary osteoarthritis of the hand" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2012-06-15T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:305e9f59-1dc6-48d2-8c7a-77e4f4804555", - "resource": { - "resourceType": "Observation", - "id": "305e9f59-1dc6-48d2-8c7a-77e4f4804555", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" - }, - "effectiveDateTime": "2012-06-15T12:00:58-07:00", - "issued": "2012-06-15T12:00:58.308-07:00", - "valueQuantity": { - "value": 7, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6bd4f119-9abb-4a22-8f2f-ac781b6caa0b", - "resource": { - "resourceType": "MedicationOrder", - "id": "6bd4f119-9abb-4a22-8f2f-ac781b6caa0b", - "dateWritten": "2012-06-15T12:00:58-07:00", - "status": "active", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "prescriber": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - }, - "encounter": { - "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" - }, - "reasonReference": { - "reference": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "849574", - "display": "Naproxen sodium 220 MG Oral Tablet" - } - ], - "text": "Naproxen sodium 220 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:241322e6-6258-4f8f-bb9b-7db50ea1d724", - "resource": { - "resourceType": "Claim", - "id": "241322e6-6258-4f8f-bb9b-7db50ea1d724", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:6bd4f119-9abb-4a22-8f2f-ac781b6caa0b" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f842512a-bab8-4a84-b5ee-1326d9123818", - "resource": { - "resourceType": "CarePlan", - "id": "f842512a-bab8-4a84-b5ee-1326d9123818", - "text": { - "status": "generated", - "div": "
    Musculoskeletal care
    " - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "active", - "context": { - "reference": "urn:uuid:1b443fd6-bc4a-4ed2-b87f-ce363e5a59f4" - }, - "period": { - "start": "2012-06-15T12:00:58-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "408869004", - "display": "Musculoskeletal care" - } - ], - "text": "Musculoskeletal care" - } - ], - "addresses": [ - { - "reference": "urn:uuid:1dd1524c-d3bd-47d6-b99e-f51d62988a68" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "386294003", - "display": "Joint mobility exercises" - } - ], - "text": "Joint mobility exercises" - }, - "status": "in-progress", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266694003", - "display": "Heat therapy" - } - ], - "text": "Heat therapy" - }, - "status": "in-progress", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:d45e2e6e-10ed-4121-a429-9f4c15b74947", - "resource": { - "resourceType": "Claim", - "id": "d45e2e6e-10ed-4121-a429-9f4c15b74947", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "201834006", - "display": "Localized, primary osteoarthritis of the hand" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395", - "resource": { - "resourceType": "Encounter", - "id": "240e4ed8-ade1-44d5-86ef-4ea380cdf395", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2012-07-04T12:00:58-07:00", - "end": "2012-07-04T12:15:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:38f326e5-1dec-4a39-8995-b4064daf5f03", - "resource": { - "resourceType": "Observation", - "id": "38f326e5-1dec-4a39-8995-b4064daf5f03", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a8a88ac-bdb8-4a49-ad37-7a52131108ea", - "resource": { - "resourceType": "Observation", - "id": "5a8a88ac-bdb8-4a49-ad37-7a52131108ea", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:540fee28-8237-4115-86b4-562c107043d5", - "resource": { - "resourceType": "Observation", - "id": "540fee28-8237-4115-86b4-562c107043d5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd091d9d-c5d3-479e-9814-8afead7037e3", - "resource": { - "resourceType": "Observation", - "id": "dd091d9d-c5d3-479e-9814-8afead7037e3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d23a7ab-eede-4abb-aa68-b7f21ca5a322", - "resource": { - "resourceType": "Observation", - "id": "3d23a7ab-eede-4abb-aa68-b7f21ca5a322", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 71, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 137, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:218256f0-6bc4-46d8-9843-bbd0920737cf", - "resource": { - "resourceType": "Observation", - "id": "218256f0-6bc4-46d8-9843-bbd0920737cf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 99, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc6a4e2e-9bd3-469a-9197-1530712c795c", - "resource": { - "resourceType": "Observation", - "id": "fc6a4e2e-9bd3-469a-9197-1530712c795c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8593ac1e-1a4a-4672-aaae-0019a347b367", - "resource": { - "resourceType": "Observation", - "id": "8593ac1e-1a4a-4672-aaae-0019a347b367", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - }, - "effectiveDateTime": "2012-07-04T12:00:58-07:00", - "issued": "2012-07-04T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:02f0cbee-2bd9-4c49-9fc4-9e0c9463c9e0", - "resource": { - "resourceType": "Immunization", - "id": "02f0cbee-2bd9-4c49-9fc4-9e0c9463c9e0", - "status": "completed", - "date": "2012-07-04T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a652552c-9cc4-4354-825c-3e22e12f79e6", - "resource": { - "resourceType": "Immunization", - "id": "a652552c-9cc4-4354-825c-3e22e12f79e6", - "status": "completed", - "date": "2012-07-04T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:abb57b3a-9584-45fd-8152-4d6fb475fed8", - "resource": { - "resourceType": "Immunization", - "id": "abb57b3a-9584-45fd-8152-4d6fb475fed8", - "status": "completed", - "date": "2012-07-04T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:240e4ed8-ade1-44d5-86ef-4ea380cdf395" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:01360dba-1b28-43ed-af0a-d2754be4daa4", - "resource": { - "resourceType": "Claim", - "id": "01360dba-1b28-43ed-af0a-d2754be4daa4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634", - "resource": { - "resourceType": "Encounter", - "id": "083255a3-e7ff-47e7-b19f-12762d8c5634", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2013-07-10T12:00:58-07:00", - "end": "2013-07-10T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b4668fce-f765-4ec1-86ab-78a78f35473b", - "resource": { - "resourceType": "Observation", - "id": "b4668fce-f765-4ec1-86ab-78a78f35473b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:600defb2-666b-431d-a51c-0eec2aa8bb54", - "resource": { - "resourceType": "Observation", - "id": "600defb2-666b-431d-a51c-0eec2aa8bb54", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e6853fa-5ed1-45f6-9529-70b0e576def0", - "resource": { - "resourceType": "Observation", - "id": "3e6853fa-5ed1-45f6-9529-70b0e576def0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0721f1f3-f563-4c09-81d6-7af15c8b57c0", - "resource": { - "resourceType": "Observation", - "id": "0721f1f3-f563-4c09-81d6-7af15c8b57c0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aff7f7a1-502d-4e63-a1ee-1f9fec6b2e3a", - "resource": { - "resourceType": "Observation", - "id": "aff7f7a1-502d-4e63-a1ee-1f9fec6b2e3a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 90, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 109, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f2e4a481-2fe2-4b7d-85a5-adabfea77016", - "resource": { - "resourceType": "Observation", - "id": "f2e4a481-2fe2-4b7d-85a5-adabfea77016", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 95, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f7155a03-508f-4960-8138-4eeb444b5930", - "resource": { - "resourceType": "Observation", - "id": "f7155a03-508f-4960-8138-4eeb444b5930", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76b8c2c5-c04b-45a6-aa27-b805c92b286d", - "resource": { - "resourceType": "Observation", - "id": "76b8c2c5-c04b-45a6-aa27-b805c92b286d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 7.6507, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4bc7ed46-4ae8-4675-8947-ab9a54702eb4", - "resource": { - "resourceType": "Observation", - "id": "4bc7ed46-4ae8-4675-8947-ab9a54702eb4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 4.1549, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3633b3f9-8a85-40ff-ab55-6af7bf2b12c1", - "resource": { - "resourceType": "Observation", - "id": "3633b3f9-8a85-40ff-ab55-6af7bf2b12c1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 12.313, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db3d344a-fd48-42a0-b9a5-6f795324d1af", - "resource": { - "resourceType": "Observation", - "id": "db3d344a-fd48-42a0-b9a5-6f795324d1af", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 39.705, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1eb8ca1-7770-4829-b9c1-596846764e0f", - "resource": { - "resourceType": "Observation", - "id": "a1eb8ca1-7770-4829-b9c1-596846764e0f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 89.137, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bad012c-757e-415e-985f-9d682c677544", - "resource": { - "resourceType": "Observation", - "id": "7bad012c-757e-415e-985f-9d682c677544", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.554, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25c86423-01b5-438d-b1f1-89d3b68794ea", - "resource": { - "resourceType": "Observation", - "id": "25c86423-01b5-438d-b1f1-89d3b68794ea", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 34.551, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddc16784-c507-4108-a6d8-cae0b05ba254", - "resource": { - "resourceType": "Observation", - "id": "ddc16784-c507-4108-a6d8-cae0b05ba254", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 42.072, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db9f82bd-802d-499c-8217-0169a0fe0339", - "resource": { - "resourceType": "Observation", - "id": "db9f82bd-802d-499c-8217-0169a0fe0339", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 384.17, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:788de2cc-46d5-4303-a720-31c5555b0578", - "resource": { - "resourceType": "Observation", - "id": "788de2cc-46d5-4303-a720-31c5555b0578", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 514.21, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76cd5814-68d0-4675-8792-418d633702f7", - "resource": { - "resourceType": "Observation", - "id": "76cd5814-68d0-4675-8792-418d633702f7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueQuantity": { - "value": 11.796, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a037d03b-3b1d-439e-b16f-295b9f31e79d", - "resource": { - "resourceType": "Observation", - "id": "a037d03b-3b1d-439e-b16f-295b9f31e79d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2a86e052-4200-499e-99e7-48f56bea808d", - "resource": { - "resourceType": "Procedure", - "id": "2a86e052-4200-499e-99e7-48f56bea808d", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2013-07-10T12:00:58-07:00", - "end": "2013-07-10T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:6dfdb7aa-3bd7-41db-96e1-5017516c6bc5", - "resource": { - "resourceType": "Immunization", - "id": "6dfdb7aa-3bd7-41db-96e1-5017516c6bc5", - "status": "completed", - "date": "2013-07-10T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:41cf5440-11c5-4b5c-9026-0b6a762f7afe", - "resource": { - "resourceType": "DiagnosticReport", - "id": "41cf5440-11c5-4b5c-9026-0b6a762f7afe", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:083255a3-e7ff-47e7-b19f-12762d8c5634" - }, - "effectiveDateTime": "2013-07-10T12:00:58-07:00", - "issued": "2013-07-10T12:00:58.308-07:00", - "performer": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "result": [ - { - "reference": "urn:uuid:76cd5814-68d0-4675-8792-418d633702f7", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:4ebd6277-499d-4ad5-baf8-b65379a59708", - "resource": { - "resourceType": "Claim", - "id": "4ebd6277-499d-4ad5-baf8-b65379a59708", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 584.84, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e", - "resource": { - "resourceType": "Encounter", - "id": "79e96930-9f24-446f-b952-9183dc2f983e", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2014-01-24T11:00:58-08:00", - "end": "2014-01-24T11:15:58-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8f9bf7af-1a0b-4c97-9237-6894444ab1c5", - "resource": { - "resourceType": "Condition", - "id": "8f9bf7af-1a0b-4c97-9237-6894444ab1c5", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e" - }, - "dateRecorded": "2014-01-24", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2014-01-24T11:00:58-08:00", - "abatementDateTime": "2014-02-05T11:00:58-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:580e4c5e-240f-4160-a5ee-fdd3c10b3700", - "resource": { - "resourceType": "Observation", - "id": "580e4c5e-240f-4160-a5ee-fdd3c10b3700", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:79e96930-9f24-446f-b952-9183dc2f983e" - }, - "effectiveDateTime": "2014-01-24T11:00:58-08:00", - "issued": "2014-01-24T11:00:58.308-08:00", - "valueQuantity": { - "value": 37.233, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ae86b3f9-9e0e-42d1-909c-6576b417868f", - "resource": { - "resourceType": "Claim", - "id": "ae86b3f9-9e0e-42d1-909c-6576b417868f", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13", - "resource": { - "resourceType": "Encounter", - "id": "3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2014-07-16T12:00:58-07:00", - "end": "2014-07-16T12:15:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7fe14f9b-06ba-4484-bc45-69b00e554b21", - "resource": { - "resourceType": "Observation", - "id": "7fe14f9b-06ba-4484-bc45-69b00e554b21", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9996ad06-6867-4b1b-ab59-63dc92605a2f", - "resource": { - "resourceType": "Observation", - "id": "9996ad06-6867-4b1b-ab59-63dc92605a2f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7adc1ace-11ff-40bc-9022-aed720fa2fdf", - "resource": { - "resourceType": "Observation", - "id": "7adc1ace-11ff-40bc-9022-aed720fa2fdf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f42104dc-7a64-4a17-90f0-a5513734b62a", - "resource": { - "resourceType": "Observation", - "id": "f42104dc-7a64-4a17-90f0-a5513734b62a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71a86a92-d879-4377-9008-ec91d062de7d", - "resource": { - "resourceType": "Observation", - "id": "71a86a92-d879-4377-9008-ec91d062de7d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 137, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:370be409-7013-42cb-9735-f4e85653c888", - "resource": { - "resourceType": "Observation", - "id": "370be409-7013-42cb-9735-f4e85653c888", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 87, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ad14f7d9-d641-4b85-a4be-3a9f0c666440", - "resource": { - "resourceType": "Observation", - "id": "ad14f7d9-d641-4b85-a4be-3a9f0c666440", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e081309e-3f38-4965-8f1e-bdc8ee35cfe8", - "resource": { - "resourceType": "Observation", - "id": "e081309e-3f38-4965-8f1e-bdc8ee35cfe8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 164.71, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e60fb867-b6d9-43f6-8b8d-6bfa67b82efd", - "resource": { - "resourceType": "Observation", - "id": "e60fb867-b6d9-43f6-8b8d-6bfa67b82efd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 134.36, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:325e8e02-fde0-45b1-b89b-0842612852db", - "resource": { - "resourceType": "Observation", - "id": "325e8e02-fde0-45b1-b89b-0842612852db", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 58.020, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a117c30-470c-42e9-83fd-751207b99985", - "resource": { - "resourceType": "Observation", - "id": "3a117c30-470c-42e9-83fd-751207b99985", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueQuantity": { - "value": 79.810, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:00a7ebab-6270-43b4-b31a-82bc49456473", - "resource": { - "resourceType": "Observation", - "id": "00a7ebab-6270-43b4-b31a-82bc49456473", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:515bb603-b614-433d-92ab-a33aa592a49f", - "resource": { - "resourceType": "Immunization", - "id": "515bb603-b614-433d-92ab-a33aa592a49f", - "status": "completed", - "date": "2014-07-16T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d2ee14f9-fe75-4b21-b3b6-d82e3ff156f9", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d2ee14f9-fe75-4b21-b3b6-d82e3ff156f9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3fc3b5e5-2abb-4af7-8a8b-6c68643e5a13" - }, - "effectiveDateTime": "2014-07-16T12:00:58-07:00", - "issued": "2014-07-16T12:00:58.308-07:00", - "performer": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "result": [ - { - "reference": "urn:uuid:3a117c30-470c-42e9-83fd-751207b99985", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7ae5ea9e-1779-4946-ae63-b91341a2eaa0", - "resource": { - "resourceType": "Claim", - "id": "7ae5ea9e-1779-4946-ae63-b91341a2eaa0", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a42d3289-5a4b-426f-95d7-2ebae5910f85", - "resource": { - "resourceType": "Encounter", - "id": "a42d3289-5a4b-426f-95d7-2ebae5910f85", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2014-07-24T12:00:58-07:00", - "end": "2014-07-24T12:15:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4106bd3e-bcf0-4870-bb18-ed13760df7a4", - "resource": { - "resourceType": "Condition", - "id": "4106bd3e-bcf0-4870-bb18-ed13760df7a4", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:a42d3289-5a4b-426f-95d7-2ebae5910f85" - }, - "dateRecorded": "2014-07-24", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2014-07-24T12:00:58-07:00", - "abatementDateTime": "2014-07-31T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f4153721-c628-445e-ad24-31c30916f66c", - "resource": { - "resourceType": "Claim", - "id": "f4153721-c628-445e-ad24-31c30916f66c", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7", - "resource": { - "resourceType": "Encounter", - "id": "3d768562-15a0-4a0f-b285-b9dae7e085f7", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2015-07-22T12:00:58-07:00", - "end": "2015-07-22T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e54762e7-1536-4237-addf-293f9ac683b3", - "resource": { - "resourceType": "Observation", - "id": "e54762e7-1536-4237-addf-293f9ac683b3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36ce9719-9fb3-46f6-91dd-ccf5a53a5f04", - "resource": { - "resourceType": "Observation", - "id": "36ce9719-9fb3-46f6-91dd-ccf5a53a5f04", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db948571-13c2-444a-98a9-cd18dc2bb9ae", - "resource": { - "resourceType": "Observation", - "id": "db948571-13c2-444a-98a9-cd18dc2bb9ae", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d0ac5208-5945-4e36-ac28-9ac4b0dae76b", - "resource": { - "resourceType": "Observation", - "id": "d0ac5208-5945-4e36-ac28-9ac4b0dae76b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6fe412e7-3a60-46fd-a173-9006ab260614", - "resource": { - "resourceType": "Observation", - "id": "6fe412e7-3a60-46fd-a173-9006ab260614", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a32447e4-79cc-45f9-9798-c650dafd469b", - "resource": { - "resourceType": "Observation", - "id": "a32447e4-79cc-45f9-9798-c650dafd469b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 100, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1aef7b5-9a41-4028-95cd-070ecda7c4ad", - "resource": { - "resourceType": "Observation", - "id": "f1aef7b5-9a41-4028-95cd-070ecda7c4ad", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:434f732e-1a60-4b46-a4af-f16bb1831bbf", - "resource": { - "resourceType": "Observation", - "id": "434f732e-1a60-4b46-a4af-f16bb1831bbf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - }, - "effectiveDateTime": "2015-07-22T12:00:58-07:00", - "issued": "2015-07-22T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a8e2d0d4-b7a8-460a-8113-cff272adc57b", - "resource": { - "resourceType": "Procedure", - "id": "a8e2d0d4-b7a8-460a-8113-cff272adc57b", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2015-07-22T12:00:58-07:00", - "end": "2015-07-22T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5ff95e5d-e7bb-4bfd-9596-80aaef6c0d2b", - "resource": { - "resourceType": "Immunization", - "id": "5ff95e5d-e7bb-4bfd-9596-80aaef6c0d2b", - "status": "completed", - "date": "2015-07-22T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3d768562-15a0-4a0f-b285-b9dae7e085f7" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4639e7e4-1217-4adc-904f-36a0de124546", - "resource": { - "resourceType": "Claim", - "id": "4639e7e4-1217-4adc-904f-36a0de124546", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 396.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:567cba0f-6094-40f3-997e-85ae2e6c810b", - "resource": { - "resourceType": "Encounter", - "id": "567cba0f-6094-40f3-997e-85ae2e6c810b", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2016-06-27T12:00:58-07:00", - "end": "2016-06-27T12:40:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6cd1c759-73c5-4293-830f-70d2f6aaf262", - "resource": { - "resourceType": "Procedure", - "id": "6cd1c759-73c5-4293-830f-70d2f6aaf262", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "performedPeriod": { - "start": "2016-06-27T12:00:58-07:00", - "end": "2016-06-27T12:25:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:567cba0f-6094-40f3-997e-85ae2e6c810b" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c4bb412b-a76e-4270-ad57-2ff178f77ac3", - "resource": { - "resourceType": "Claim", - "id": "c4bb412b-a76e-4270-ad57-2ff178f77ac3", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - }, - "net": { - "value": 16189.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99", - "resource": { - "resourceType": "Encounter", - "id": "c8931c16-5d87-4d20-83bd-65e1e90fbd99", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2016-07-27T12:00:58-07:00", - "end": "2016-07-27T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f9eba7ef-b8d9-4d5b-bd54-3e5d34d18b1b", - "resource": { - "resourceType": "Observation", - "id": "f9eba7ef-b8d9-4d5b-bd54-3e5d34d18b1b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b9e8a11-50d9-4d70-b635-c8646b15b444", - "resource": { - "resourceType": "Observation", - "id": "1b9e8a11-50d9-4d70-b635-c8646b15b444", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ae5e618d-9ed5-4a8f-8812-562a422e9dc3", - "resource": { - "resourceType": "Observation", - "id": "ae5e618d-9ed5-4a8f-8812-562a422e9dc3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:745edee9-abef-4ab6-bd47-e50b9e0e96bc", - "resource": { - "resourceType": "Observation", - "id": "745edee9-abef-4ab6-bd47-e50b9e0e96bc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6f8d163-9560-4696-875e-222bc5d9599f", - "resource": { - "resourceType": "Observation", - "id": "e6f8d163-9560-4696-875e-222bc5d9599f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 107, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebf6d0e7-e45c-4109-a2aa-c0d9e43b03b0", - "resource": { - "resourceType": "Observation", - "id": "ebf6d0e7-e45c-4109-a2aa-c0d9e43b03b0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 70, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9fc136c1-6ae9-4c6b-b429-0f228a7bf488", - "resource": { - "resourceType": "Observation", - "id": "9fc136c1-6ae9-4c6b-b429-0f228a7bf488", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8f7b5bdf-4659-49ca-a58a-89b90869da05", - "resource": { - "resourceType": "Observation", - "id": "8f7b5bdf-4659-49ca-a58a-89b90869da05", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - }, - "effectiveDateTime": "2016-07-27T12:00:58-07:00", - "issued": "2016-07-27T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e3a78a6-3c18-45fe-9988-c6517a14697c", - "resource": { - "resourceType": "Procedure", - "id": "5e3a78a6-3c18-45fe-9988-c6517a14697c", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2016-07-27T12:00:58-07:00", - "end": "2016-07-27T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e57ffcef-b89a-4b77-8bd0-90245c77b94e", - "resource": { - "resourceType": "Immunization", - "id": "e57ffcef-b89a-4b77-8bd0-90245c77b94e", - "status": "completed", - "date": "2016-07-27T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:c8931c16-5d87-4d20-83bd-65e1e90fbd99" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:cb1ebce3-7b75-40dd-94e7-91ca42903108", - "resource": { - "resourceType": "Claim", - "id": "cb1ebce3-7b75-40dd-94e7-91ca42903108", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 647.18, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39", - "resource": { - "resourceType": "Encounter", - "id": "4a5a04f3-96ae-4122-9a06-48e5a6af8a39", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2017-08-02T12:00:58-07:00", - "end": "2017-08-02T12:15:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:24daf66a-4e5b-4ed4-a2ad-e681bab296fd", - "resource": { - "resourceType": "Observation", - "id": "24daf66a-4e5b-4ed4-a2ad-e681bab296fd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e494d610-29fc-4aa5-80a3-2d17c5070303", - "resource": { - "resourceType": "Observation", - "id": "e494d610-29fc-4aa5-80a3-2d17c5070303", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ab1a0f5-97fb-4774-827b-2e2ef9ff202b", - "resource": { - "resourceType": "Observation", - "id": "9ab1a0f5-97fb-4774-827b-2e2ef9ff202b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b53552c4-019f-433f-a611-81138162a747", - "resource": { - "resourceType": "Observation", - "id": "b53552c4-019f-433f-a611-81138162a747", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bb9ee043-e1d4-4f14-a95e-2d39337b60f1", - "resource": { - "resourceType": "Observation", - "id": "bb9ee043-e1d4-4f14-a95e-2d39337b60f1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0640cdf1-ef7b-42f8-8d0d-3689a7cc8e78", - "resource": { - "resourceType": "Observation", - "id": "0640cdf1-ef7b-42f8-8d0d-3689a7cc8e78", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 78, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:440a4598-a8ff-43ac-a7a9-d5f99c4b56cb", - "resource": { - "resourceType": "Observation", - "id": "440a4598-a8ff-43ac-a7a9-d5f99c4b56cb", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0216a17a-f1d8-4c76-884a-8a2183878030", - "resource": { - "resourceType": "Observation", - "id": "0216a17a-f1d8-4c76-884a-8a2183878030", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 163.89, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:10fae0a3-57f5-408e-967b-3c6a77304419", - "resource": { - "resourceType": "Observation", - "id": "10fae0a3-57f5-408e-967b-3c6a77304419", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 116.74, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3669070-1cc6-48af-9029-f23b3a690767", - "resource": { - "resourceType": "Observation", - "id": "e3669070-1cc6-48af-9029-f23b3a690767", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 71.120, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7c490c0-238b-4f3c-a6e3-e796a356ad3e", - "resource": { - "resourceType": "Observation", - "id": "b7c490c0-238b-4f3c-a6e3-e796a356ad3e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueQuantity": { - "value": 69.430, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ce286d4-a2df-4f98-ae0c-14db797963d9", - "resource": { - "resourceType": "Observation", - "id": "5ce286d4-a2df-4f98-ae0c-14db797963d9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:79557d6a-cdc0-4988-9fd7-2df640bdbc16", - "resource": { - "resourceType": "Immunization", - "id": "79557d6a-cdc0-4988-9fd7-2df640bdbc16", - "status": "completed", - "date": "2017-08-02T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1f9c7067-6385-4093-8a47-eb796fc74561", - "resource": { - "resourceType": "DiagnosticReport", - "id": "1f9c7067-6385-4093-8a47-eb796fc74561", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:4a5a04f3-96ae-4122-9a06-48e5a6af8a39" - }, - "effectiveDateTime": "2017-08-02T12:00:58-07:00", - "issued": "2017-08-02T12:00:58.308-07:00", - "performer": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "result": [ - { - "reference": "urn:uuid:b7c490c0-238b-4f3c-a6e3-e796a356ad3e", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:2de52b8d-448f-4d64-ba18-dd8c5a167a3e", - "resource": { - "resourceType": "Claim", - "id": "2de52b8d-448f-4d64-ba18-dd8c5a167a3e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635", - "resource": { - "resourceType": "Encounter", - "id": "68ea2cc6-d2b0-460c-9724-1b8e9fc17635", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - } - } - ], - "period": { - "start": "2018-04-18T12:00:58-07:00", - "end": "2018-04-18T12:30:58-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b", - "resource": { - "resourceType": "Condition", - "id": "333435c4-b72c-4f42-a672-8861806c7c5b", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" - }, - "dateRecorded": "2018-04-18", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2018-04-18T12:00:58-07:00", - "abatementDateTime": "2018-04-25T12:00:58-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:aaff4b8b-ab0a-4941-be2a-b093e29eef8e", - "resource": { - "resourceType": "Observation", - "id": "aaff4b8b-ab0a-4941-be2a-b093e29eef8e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" - }, - "effectiveDateTime": "2018-04-18T12:00:58-07:00", - "issued": "2018-04-18T12:00:58.308-07:00", - "valueQuantity": { - "value": 38.525, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ad28370-2ab5-42e7-896c-e5bacc7fbc79", - "resource": { - "resourceType": "Procedure", - "id": "5ad28370-2ab5-42e7-896c-e5bacc7fbc79", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "reasonReference": { - "reference": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b" - }, - "performedPeriod": { - "start": "2018-04-18T12:00:58-07:00", - "end": "2018-04-18T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:8382a2bb-e319-40eb-ab05-98a73700a74f", - "resource": { - "resourceType": "MedicationOrder", - "id": "8382a2bb-e319-40eb-ab05-98a73700a74f", - "dateWritten": "2018-04-18T12:00:58-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "prescriber": { - "reference": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203" - }, - "encounter": { - "reference": "urn:uuid:68ea2cc6-d2b0-460c-9724-1b8e9fc17635" - }, - "reasonReference": { - "reference": "urn:uuid:333435c4-b72c-4f42-a672-8861806c7c5b" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "834102", - "display": "Penicillin V Potassium 500 MG Oral Tablet" - } - ], - "text": "Penicillin V Potassium 500 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:db44cd30-b644-41b1-ad33-da4655ed8912", - "resource": { - "resourceType": "Claim", - "id": "db44cd30-b644-41b1-ad33-da4655ed8912", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:8382a2bb-e319-40eb-ab05-98a73700a74f" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d296fe0d-9770-4382-a2c3-814a59e19249", - "resource": { - "resourceType": "Claim", - "id": "d296fe0d-9770-4382-a2c3-814a59e19249", - "type": "institutional", - "organization": { - "reference": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - }, - "net": { - "value": 3449.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be", - "resource": { - "resourceType": "Encounter", - "id": "ba203c7a-efb5-4008-ba3e-12aacf75b7be", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2018-08-08T12:00:58-07:00", - "end": "2018-08-08T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c11057bb-b0d8-4e39-84e6-0f0d3ef5da44", - "resource": { - "resourceType": "Observation", - "id": "c11057bb-b0d8-4e39-84e6-0f0d3ef5da44", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e768f70e-bf68-40ff-bd42-ef89773fca31", - "resource": { - "resourceType": "Observation", - "id": "e768f70e-bf68-40ff-bd42-ef89773fca31", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bd79284-fed8-47b1-9add-e51d61aca0fc", - "resource": { - "resourceType": "Observation", - "id": "7bd79284-fed8-47b1-9add-e51d61aca0fc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c65c5c6-e61b-4087-abab-2996a5ab4802", - "resource": { - "resourceType": "Observation", - "id": "9c65c5c6-e61b-4087-abab-2996a5ab4802", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ad76fead-9f21-4bc2-9c9d-3bdd08cdd4ef", - "resource": { - "resourceType": "Observation", - "id": "ad76fead-9f21-4bc2-9c9d-3bdd08cdd4ef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 89, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 112, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d40ab67c-b7f7-4d1c-a09c-5214286ddf03", - "resource": { - "resourceType": "Observation", - "id": "d40ab67c-b7f7-4d1c-a09c-5214286ddf03", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 88, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:910e9488-2c61-4290-97ca-e3205c9f32f1", - "resource": { - "resourceType": "Observation", - "id": "910e9488-2c61-4290-97ca-e3205c9f32f1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:daa89c5e-116d-4445-9ce6-586bc5551edb", - "resource": { - "resourceType": "Observation", - "id": "daa89c5e-116d-4445-9ce6-586bc5551edb", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 3.8248, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:de2d496b-efff-4706-b4ee-798a7e91b01b", - "resource": { - "resourceType": "Observation", - "id": "de2d496b-efff-4706-b4ee-798a7e91b01b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 4.3428, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ae49c09-8014-48b5-902b-b03e1d1e184d", - "resource": { - "resourceType": "Observation", - "id": "1ae49c09-8014-48b5-902b-b03e1d1e184d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 12.671, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d29d9dce-74eb-43c5-85a2-b9788dd25854", - "resource": { - "resourceType": "Observation", - "id": "d29d9dce-74eb-43c5-85a2-b9788dd25854", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 46.024, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1531a503-62f1-4cca-95d2-f4bf159ac353", - "resource": { - "resourceType": "Observation", - "id": "1531a503-62f1-4cca-95d2-f4bf159ac353", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 83.500, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a2546a4-f65b-4239-b2f7-f453ca6e6ade", - "resource": { - "resourceType": "Observation", - "id": "3a2546a4-f65b-4239-b2f7-f453ca6e6ade", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 30.750, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51c606b1-6fab-4804-bcd7-c3375755ad6a", - "resource": { - "resourceType": "Observation", - "id": "51c606b1-6fab-4804-bcd7-c3375755ad6a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 34.274, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:18bab09d-0472-4686-aeba-a9c2783666a4", - "resource": { - "resourceType": "Observation", - "id": "18bab09d-0472-4686-aeba-a9c2783666a4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 43.191, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ccefee54-1a5a-40ea-a83f-c9b5ff5ad92e", - "resource": { - "resourceType": "Observation", - "id": "ccefee54-1a5a-40ea-a83f-c9b5ff5ad92e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 312.26, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4f3f1787-3c93-4b8a-84c5-62ef01d4873e", - "resource": { - "resourceType": "Observation", - "id": "4f3f1787-3c93-4b8a-84c5-62ef01d4873e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 436.34, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:42965d15-3301-4c48-8d25-738b68e01dd2", - "resource": { - "resourceType": "Observation", - "id": "42965d15-3301-4c48-8d25-738b68e01dd2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueQuantity": { - "value": 12.100, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:082fb329-33f0-495f-b84f-41ac859eb949", - "resource": { - "resourceType": "Observation", - "id": "082fb329-33f0-495f-b84f-41ac859eb949", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:be6662e1-fa6d-4128-a972-7fc0954f94a4", - "resource": { - "resourceType": "Procedure", - "id": "be6662e1-fa6d-4128-a972-7fc0954f94a4", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2018-08-08T12:00:58-07:00", - "end": "2018-08-08T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9d87a700-2cf6-4f23-a413-f8bb7f663675", - "resource": { - "resourceType": "Immunization", - "id": "9d87a700-2cf6-4f23-a413-f8bb7f663675", - "status": "completed", - "date": "2018-08-08T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f091e585-428c-4782-a5da-2968701c1465", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f091e585-428c-4782-a5da-2968701c1465", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:ba203c7a-efb5-4008-ba3e-12aacf75b7be" - }, - "effectiveDateTime": "2018-08-08T12:00:58-07:00", - "issued": "2018-08-08T12:00:58.308-07:00", - "performer": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "result": [ - { - "reference": "urn:uuid:42965d15-3301-4c48-8d25-738b68e01dd2", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:2b30f498-2f90-437c-98f4-bef44474b923", - "resource": { - "resourceType": "Claim", - "id": "2b30f498-2f90-437c-98f4-bef44474b923", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 565.66, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821", - "resource": { - "resourceType": "Encounter", - "id": "3a3a2573-b9b4-4889-bca7-741859f50821", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240" - } - } - ], - "period": { - "start": "2019-08-14T12:00:58-07:00", - "end": "2019-08-14T12:30:58-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ab6bc4b0-b00a-46ea-8b1c-d5a705984aec", - "resource": { - "resourceType": "Observation", - "id": "ab6bc4b0-b00a-46ea-8b1c-d5a705984aec", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 167.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ce77c79-b222-4514-aa61-502863f98fa7", - "resource": { - "resourceType": "Observation", - "id": "1ce77c79-b222-4514-aa61-502863f98fa7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f40c57fd-3980-4812-8717-f7485e845269", - "resource": { - "resourceType": "Observation", - "id": "f40c57fd-3980-4812-8717-f7485e845269", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 78.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e063568-0f08-44d6-86b7-049deac8d2e2", - "resource": { - "resourceType": "Observation", - "id": "1e063568-0f08-44d6-86b7-049deac8d2e2", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 27.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:13b4ec52-110c-4f69-8de7-fc748c7f878b", - "resource": { - "resourceType": "Observation", - "id": "13b4ec52-110c-4f69-8de7-fc748c7f878b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 123, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebf25abe-a307-4b91-a440-e5e79287a97a", - "resource": { - "resourceType": "Observation", - "id": "ebf25abe-a307-4b91-a440-e5e79287a97a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a095c10f-4697-4b7d-ac78-db87fc120746", - "resource": { - "resourceType": "Observation", - "id": "a095c10f-4697-4b7d-ac78-db87fc120746", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eb5cfa3d-6f2b-46d6-b8c0-dccd54a1d296", - "resource": { - "resourceType": "Observation", - "id": "eb5cfa3d-6f2b-46d6-b8c0-dccd54a1d296", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - }, - "effectiveDateTime": "2019-08-14T12:00:58-07:00", - "issued": "2019-08-14T12:00:58.308-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f18047c-a076-4cba-a2d7-90027a3f2254", - "resource": { - "resourceType": "Procedure", - "id": "5f18047c-a076-4cba-a2d7-90027a3f2254", - "subject": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2019-08-14T12:00:58-07:00", - "end": "2019-08-14T12:15:58-07:00" - }, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:fce25c30-850e-481a-a82a-6e6c1220d2a1", - "resource": { - "resourceType": "Immunization", - "id": "fce25c30-850e-481a-a82a-6e6c1220d2a1", - "status": "completed", - "date": "2019-08-14T12:00:58-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:3a3a2573-b9b4-4889-bca7-741859f50821" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:543426ea-0bfc-4402-8f95-ccd34b7b6ddd", - "resource": { - "resourceType": "Claim", - "id": "543426ea-0bfc-4402-8f95-ccd34b7b6ddd", - "type": "institutional", - "organization": { - "reference": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:8ed31d3e-7352-4055-89c4-f017db3f594e" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 619.62, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json deleted file mode 100644 index 3a6ae7a6d914..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/Terry864_Hintz995_f04ef974-6d6f-4e1c-804f-cd4d62aabb4f.json +++ /dev/null @@ -1,10047 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f", - "resource": { - "resourceType": "Patient", - "id": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7196695550560093073 Population seed: 1586309754086
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-race", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Race", - "code": "2106-3", - "display": "White" - } - ], - "text": "White" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/us-core-ethnicity", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/Ethnicity", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - ], - "text": "Not Hispanic or Latino" - } - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Daniele610 Zemlak964" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "M" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Ashburnham", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.6685442920261035 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 48.331455707973895 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR" - } - ] - }, - "system": "http://hospital.smarthealthit.org", - "value": "f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB" - } - ] - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-11-6709" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "DL" - } - ] - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99940409" - } - ], - "name": [ - { - "use": "official", - "family": [ - "Hintz995" - ], - "given": [ - "Terry864" - ], - "prefix": [ - "Mr." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-451-4282", - "use": "home" - } - ], - "gender": "male", - "birthDate": "1970-06-15", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.305010719394204 - }, - { - "url": "longitude", - "valueDecimal": -72.70463521406347 - } - ] - } - ], - "line": [ - "718 Toy Parade" - ], - "city": "Easthampton", - "state": "Massachusetts", - "postalCode": "01027", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "M" - } - ] - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", - "resource": { - "resourceType": "Organization", - "id": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "PCP15859", - "telecom": [ - { - "system": "phone", - "value": "413-387-2118" - } - ], - "address": [ - { - "line": [ - "92 MAIN ST" - ], - "city": "FLORENCE", - "state": "MA", - "postalCode": "01062-1460", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd", - "resource": { - "resourceType": "Practitioner", - "id": "6d5162ad-3294-398e-9eb4-7021465d4bcd", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10930" - } - ], - "active": true, - "name": { - "family": [ - "Pfeffer420" - ], - "given": [ - "Darryl392" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "92 MAIN ST" - ], - "city": "FLORENCE", - "state": "MA", - "postalCode": "01062-1460", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:2a908737-0281-49dc-be45-550d71d74d25", - "resource": { - "resourceType": "Encounter", - "id": "2a908737-0281-49dc-be45-550d71d74d25", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "1998-08-24T15:14:32-07:00", - "end": "1998-08-24T15:44:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e494cb27-116c-46ed-9ed5-250cfb3c86e6", - "resource": { - "resourceType": "Condition", - "id": "e494cb27-116c-46ed-9ed5-250cfb3c86e6", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:2a908737-0281-49dc-be45-550d71d74d25" - }, - "dateRecorded": "1998-08-24", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "1998-08-24T15:14:32-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b0433c04-7930-4449-8442-e6523198d4bc", - "resource": { - "resourceType": "Claim", - "id": "b0433c04-7930-4449-8442-e6523198d4bc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12", - "resource": { - "resourceType": "Organization", - "id": "49318f80-bd8b-3fc7-a096-ac43088b0c12", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "COOLEY DICKINSON HOSPITAL INC,THE", - "telecom": [ - { - "system": "phone", - "value": "4135822000" - } - ], - "address": [ - { - "line": [ - "30 LOCUST STREET" - ], - "city": "NORTHAMPTON", - "state": "MA", - "postalCode": "01060", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7", - "resource": { - "resourceType": "Practitioner", - "id": "c1a3f738-c767-30ab-951c-5c60f3c569c7", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "60" - } - ], - "active": true, - "name": { - "family": [ - "Hermiston71" - ], - "given": [ - "Andrea7" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "30 LOCUST STREET" - ], - "city": "NORTHAMPTON", - "state": "MA", - "postalCode": "01060", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:0de358a5-2910-44b5-8b86-2154eb0e0fba", - "resource": { - "resourceType": "Encounter", - "id": "0de358a5-2910-44b5-8b86-2154eb0e0fba", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2001-04-16T15:14:32-07:00", - "end": "2012-04-30T15:14:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c734c612-92d9-4ba1-9c16-0e59dcde5f25", - "resource": { - "resourceType": "Claim", - "id": "c734c612-92d9-4ba1-9c16-0e59dcde5f25", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969", - "resource": { - "resourceType": "Encounter", - "id": "a4b4fba9-cf37-4c74-91f3-e820bf402969", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2004-08-30T15:14:32-07:00", - "end": "2004-08-30T15:29:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a3a299cc-81e0-4637-ad2f-dc392046e5dd", - "resource": { - "resourceType": "Condition", - "id": "a3a299cc-81e0-4637-ad2f-dc392046e5dd", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" - }, - "dateRecorded": "2004-08-30", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2004-08-30T15:14:32-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:bddfe782-62ba-494b-a368-bf70abb1a0c3", - "resource": { - "resourceType": "Condition", - "id": "bddfe782-62ba-494b-a368-bf70abb1a0c3", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" - }, - "dateRecorded": "2004-08-30", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "onsetDateTime": "2004-08-30T15:14:32-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:018ae8c8-863f-4b7a-b101-15966d2d926f", - "resource": { - "resourceType": "CarePlan", - "id": "018ae8c8-863f-4b7a-b101-15966d2d926f", - "text": { - "status": "generated", - "div": "
    Diabetes self management plan
    " - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "active", - "context": { - "reference": "urn:uuid:a4b4fba9-cf37-4c74-91f3-e820bf402969" - }, - "period": { - "start": "2004-08-30T15:14:32-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698360004", - "display": "Diabetes self management plan" - } - ], - "text": "Diabetes self management plan" - } - ], - "addresses": [ - { - "reference": "urn:uuid:a3a299cc-81e0-4637-ad2f-dc392046e5dd" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "160670007", - "display": "Diabetic diet" - } - ], - "text": "Diabetic diet" - }, - "status": "in-progress", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:e0f3e870-fdb7-45e3-ad22-e13879ee9743", - "resource": { - "resourceType": "Claim", - "id": "e0f3e870-fdb7-45e3-ad22-e13879ee9743", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - }, - { - "sequence": 2, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:58764f28-1efe-463c-b978-747589b6978b", - "resource": { - "resourceType": "Encounter", - "id": "58764f28-1efe-463c-b978-747589b6978b", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2004-08-30T15:14:32-07:00", - "end": "2004-08-30T16:36:32-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:00347e04-8907-43f6-a90d-03c8af72cd5c", - "resource": { - "resourceType": "MedicationOrder", - "id": "00347e04-8907-43f6-a90d-03c8af72cd5c", - "dateWritten": "2004-08-30T15:14:32-07:00", - "status": "active", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "prescriber": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - }, - "encounter": { - "reference": "urn:uuid:58764f28-1efe-463c-b978-747589b6978b" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "2001499", - "display": "Vitamin B 12 5 MG/ML Injectable Solution" - } - ], - "text": "Vitamin B 12 5 MG/ML Injectable Solution" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:f98510c6-f134-4053-b275-03bcea081a8b", - "resource": { - "resourceType": "Claim", - "id": "f98510c6-f134-4053-b275-03bcea081a8b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:00347e04-8907-43f6-a90d-03c8af72cd5c" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:63e4f590-7837-4253-9a4f-f8a6e3694911", - "resource": { - "resourceType": "Claim", - "id": "63e4f590-7837-4253-9a4f-f8a6e3694911", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1", - "resource": { - "resourceType": "Encounter", - "id": "e8294067-7989-4465-8ec9-2e21c1018fd1", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2010-06-21T15:14:32-07:00", - "end": "2010-06-21T15:44:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:34e20d3d-e0ce-40d8-afae-023d328e8d76", - "resource": { - "resourceType": "Observation", - "id": "34e20d3d-e0ce-40d8-afae-023d328e8d76", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f2022249-5b3e-4d6e-b35c-db47533b057d", - "resource": { - "resourceType": "Observation", - "id": "f2022249-5b3e-4d6e-b35c-db47533b057d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2164fb5d-94f5-4e0a-a99a-ef8ee747851f", - "resource": { - "resourceType": "Observation", - "id": "2164fb5d-94f5-4e0a-a99a-ef8ee747851f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 94.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e0ebed81-17a7-4980-88a7-cc4c7a4f14a4", - "resource": { - "resourceType": "Observation", - "id": "e0ebed81-17a7-4980-88a7-cc4c7a4f14a4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 30.140, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e31ccabb-8298-4fc0-8ff0-4f46049a03f7", - "resource": { - "resourceType": "Observation", - "id": "e31ccabb-8298-4fc0-8ff0-4f46049a03f7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:08d0c372-db13-47b5-ac4b-02c08c76c4af", - "resource": { - "resourceType": "Observation", - "id": "08d0c372-db13-47b5-ac4b-02c08c76c4af", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 87, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:608899af-a8f3-41ac-804b-41057e3e87a1", - "resource": { - "resourceType": "Observation", - "id": "608899af-a8f3-41ac-804b-41057e3e87a1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bea5fe32-a356-42f2-9c96-9fc58b2581cd", - "resource": { - "resourceType": "Observation", - "id": "bea5fe32-a356-42f2-9c96-9fc58b2581cd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 80.840, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc6276d2-1abe-4887-bedf-bfb68bf72751", - "resource": { - "resourceType": "Observation", - "id": "dc6276d2-1abe-4887-bedf-bfb68bf72751", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 13.090, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1da7438a-81f3-4be2-b3bc-bdfcd955dd50", - "resource": { - "resourceType": "Observation", - "id": "1da7438a-81f3-4be2-b3bc-bdfcd955dd50", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.3100, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:093db9d0-960f-4d87-9e2f-1ff1e2725aff", - "resource": { - "resourceType": "Observation", - "id": "093db9d0-960f-4d87-9e2f-1ff1e2725aff", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 10.120, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:94d3fb0c-cb19-4804-bd23-a6c935c46b76", - "resource": { - "resourceType": "Observation", - "id": "94d3fb0c-cb19-4804-bd23-a6c935c46b76", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 137.54, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d74d8e7-2274-45f1-a083-c6c94d7b35aa", - "resource": { - "resourceType": "Observation", - "id": "9d74d8e7-2274-45f1-a083-c6c94d7b35aa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 4.7300, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:979ba582-8d53-4717-a2be-7bf9139a18a6", - "resource": { - "resourceType": "Observation", - "id": "979ba582-8d53-4717-a2be-7bf9139a18a6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 106.38, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:88c292bf-1c48-4971-8407-4cec469890f1", - "resource": { - "resourceType": "Observation", - "id": "88c292bf-1c48-4971-8407-4cec469890f1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 25.060, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d71f372b-60d3-4420-9dad-8b05c87d1ab1", - "resource": { - "resourceType": "Observation", - "id": "d71f372b-60d3-4420-9dad-8b05c87d1ab1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 6.5061, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dbde21c8-aa29-4126-9f5b-9ad4a7a598f0", - "resource": { - "resourceType": "Observation", - "id": "dbde21c8-aa29-4126-9f5b-9ad4a7a598f0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.0094, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6cbb1ea8-fb5e-4fef-9bd1-e7b00704df6d", - "resource": { - "resourceType": "Observation", - "id": "6cbb1ea8-fb5e-4fef-9bd1-e7b00704df6d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 14.763, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b4db91da-093d-49f7-b2bf-48aeafacf6d6", - "resource": { - "resourceType": "Observation", - "id": "b4db91da-093d-49f7-b2bf-48aeafacf6d6", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 47.219, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b79e5d17-aa3b-4832-8f32-fc3172671f3f", - "resource": { - "resourceType": "Observation", - "id": "b79e5d17-aa3b-4832-8f32-fc3172671f3f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 93.731, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:60cff5b5-e0e4-4b99-a321-e013d2dd6cef", - "resource": { - "resourceType": "Observation", - "id": "60cff5b5-e0e4-4b99-a321-e013d2dd6cef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 32.546, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:48dcb9e6-1f7a-4c66-b44e-3d1644724c3b", - "resource": { - "resourceType": "Observation", - "id": "48dcb9e6-1f7a-4c66-b44e-3d1644724c3b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 34.765, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e62e5b25-2399-442e-a717-0356e5e72f29", - "resource": { - "resourceType": "Observation", - "id": "e62e5b25-2399-442e-a717-0356e5e72f29", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 40.522, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5448a7ed-3736-4df0-8d9e-0341ca12029a", - "resource": { - "resourceType": "Observation", - "id": "5448a7ed-3736-4df0-8d9e-0341ca12029a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 261.03, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dac3b08f-1084-4642-9d86-27ab5b70135c", - "resource": { - "resourceType": "Observation", - "id": "dac3b08f-1084-4642-9d86-27ab5b70135c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 376.63, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6e51f20-9c12-44d6-b6c0-f10618710944", - "resource": { - "resourceType": "Observation", - "id": "c6e51f20-9c12-44d6-b6c0-f10618710944", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 11.045, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9fbe90e8-5196-4bd5-b4c2-2a1b5a38f44b", - "resource": { - "resourceType": "Observation", - "id": "9fbe90e8-5196-4bd5-b4c2-2a1b5a38f44b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67bd38b6-626d-45c8-8fe6-ab5312bf6f01", - "resource": { - "resourceType": "Observation", - "id": "67bd38b6-626d-45c8-8fe6-ab5312bf6f01", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.8400, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e888fbd3-265e-46de-ae6f-9d339c767486", - "resource": { - "resourceType": "Procedure", - "id": "e888fbd3-265e-46de-ae6f-9d339c767486", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2010-06-21T15:14:32-07:00", - "end": "2010-06-21T15:29:32-07:00" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e324eaa2-cb14-424b-a013-102df2c1b97e", - "resource": { - "resourceType": "Immunization", - "id": "e324eaa2-cb14-424b-a013-102df2c1b97e", - "status": "completed", - "date": "2010-06-21T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:89d4cc47-69ac-4e1f-bdc1-ff6b4a50d7c4", - "resource": { - "resourceType": "Immunization", - "id": "89d4cc47-69ac-4e1f-bdc1-ff6b4a50d7c4", - "status": "completed", - "date": "2010-06-21T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - } - ], - "text": "Hep A, adult" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:33b6483a-d39f-46ba-af42-76083f5c77e7", - "resource": { - "resourceType": "DiagnosticReport", - "id": "33b6483a-d39f-46ba-af42-76083f5c77e7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:88c292bf-1c48-4971-8407-4cec469890f1", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:8dc0a984-da94-4485-86e1-286bc6afe1df", - "resource": { - "resourceType": "DiagnosticReport", - "id": "8dc0a984-da94-4485-86e1-286bc6afe1df", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e8294067-7989-4465-8ec9-2e21c1018fd1" - }, - "effectiveDateTime": "2010-06-21T15:14:32-07:00", - "issued": "2010-06-21T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:c6e51f20-9c12-44d6-b6c0-f10618710944", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0014b041-5051-42b0-b302-557879cc3bbb", - "resource": { - "resourceType": "Claim", - "id": "0014b041-5051-42b0-b302-557879cc3bbb", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 651.64, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805", - "resource": { - "resourceType": "Encounter", - "id": "f124f66b-fa74-4122-8901-941e80326805", - "status": "finished", - "class": "emergency", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2010-12-10T14:14:32-08:00", - "end": "2010-12-10T16:20:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961", - "resource": { - "resourceType": "Condition", - "id": "e13c18a8-21c1-48ba-bb7a-a6b1c2cde961", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - }, - "dateRecorded": "2010-12-10", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "33737001", - "display": "Fracture of rib" - } - ], - "text": "Fracture of rib" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2010-12-10T14:14:32-08:00", - "abatementDateTime": "2011-03-10T14:14:32-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:eb7910bc-c08d-4307-be88-ace1f360436d", - "resource": { - "resourceType": "Procedure", - "id": "eb7910bc-c08d-4307-be88-ace1f360436d", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "399208008", - "display": "Chest X-ray" - } - ], - "text": "Chest X-ray" - }, - "performedPeriod": { - "start": "2010-12-10T14:14:32-08:00", - "end": "2010-12-10T14:44:32-08:00" - }, - "encounter": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f17528ad-4251-4346-9825-578a3152c169", - "resource": { - "resourceType": "Procedure", - "id": "f17528ad-4251-4346-9825-578a3152c169", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274474001", - "display": "Bone immobilization" - } - ], - "text": "Bone immobilization" - }, - "reasonReference": { - "reference": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961" - }, - "performedPeriod": { - "start": "2010-12-10T14:14:32-08:00", - "end": "2010-12-10T14:50:32-08:00" - }, - "encounter": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f5db4141-0846-48ef-8e17-6408645a4b91", - "resource": { - "resourceType": "MedicationOrder", - "id": "f5db4141-0846-48ef-8e17-6408645a4b91", - "dateWritten": "2010-12-10T14:14:32-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "prescriber": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - }, - "encounter": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "857005", - "display": "Acetaminophen 325 MG / HYDROcodone Bitartrate 7.5 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG / HYDROcodone Bitartrate 7.5 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:bf0d237a-0418-4305-8569-c6d3f05d565e", - "resource": { - "resourceType": "Claim", - "id": "bf0d237a-0418-4305-8569-c6d3f05d565e", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:f5db4141-0846-48ef-8e17-6408645a4b91" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:350364dd-9d41-478a-9370-daaa70f853ab", - "resource": { - "resourceType": "MedicationOrder", - "id": "350364dd-9d41-478a-9370-daaa70f853ab", - "dateWritten": "2010-12-10T14:14:32-08:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "prescriber": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - }, - "encounter": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "310965", - "display": "Ibuprofen 200 MG Oral Tablet" - } - ], - "text": "Ibuprofen 200 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:77bc50cc-f4cb-4c94-a483-ba156759dc46", - "resource": { - "resourceType": "Claim", - "id": "77bc50cc-f4cb-4c94-a483-ba156759dc46", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:350364dd-9d41-478a-9370-daaa70f853ab" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0fc90883-cda1-41bc-9c0a-4d10e835c8bd", - "resource": { - "resourceType": "CarePlan", - "id": "0fc90883-cda1-41bc-9c0a-4d10e835c8bd", - "text": { - "status": "generated", - "div": "
    Fracture care
    " - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "context": { - "reference": "urn:uuid:f124f66b-fa74-4122-8901-941e80326805" - }, - "period": { - "start": "2010-12-10T14:14:32-08:00", - "end": "2011-03-10T14:14:32-08:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385691007", - "display": "Fracture care" - } - ], - "text": "Fracture care" - } - ], - "addresses": [ - { - "reference": "urn:uuid:e13c18a8-21c1-48ba-bb7a-a6b1c2cde961" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "183051005", - "display": "Recommendation to rest" - } - ], - "text": "Recommendation to rest" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "408580007", - "display": "Physical activity target light exercise" - } - ], - "text": "Physical activity target light exercise" - }, - "status": "completed", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:ac92faef-c98a-4b58-8704-092977a9ae3f", - "resource": { - "resourceType": "ImagingStudy", - "id": "ac92faef-c98a-4b58-8704-092977a9ae3f", - "started": "2010-12-10T14:14:32-08:00", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "uid": "urn:oid:1.2.840.99999999.64685352.1586309774039", - "numberOfSeries": 1, - "numberOfInstances": 1, - "series": [ - { - "number": 1, - "modality": { - "system": "http://dicom.nema.org/resources/ontology/DCM", - "code": "DX", - "display": "Digital Radiography" - }, - "uid": "urn:oid:1.2.840.99999999.1.82492111.1586309774039", - "numberOfInstances": 1, - "availability": "UNAVAILABLE", - "bodySite": { - "system": "http://snomed.info/sct", - "code": "51185008", - "display": "Chest" - }, - "started": "2010-12-10T14:14:32-08:00", - "instance": [ - { - "number": 1, - "uid": "urn:oid:1.2.840.99999999.1.1.11502719.1586309774039", - "sopClass": "urn:oid:1.2.840.10008.5.1.4.1.1.1.1", - "title": "Image of chest" - } - ] - } - ] - }, - "request": { - "method": "POST", - "url": "ImagingStudy" - } - }, - { - "fullUrl": "urn:uuid:527c7821-4dab-4985-b35c-e7b39a578026", - "resource": { - "resourceType": "Claim", - "id": "527c7821-4dab-4985-b35c-e7b39a578026", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "33737001", - "display": "Fracture of rib" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "399208008", - "display": "Chest X-ray" - }, - "net": { - "value": 11005.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "274474001", - "display": "Bone immobilization" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d102fa74-8e90-4f2c-8611-11c823e3ff58", - "resource": { - "resourceType": "Encounter", - "id": "d102fa74-8e90-4f2c-8611-11c823e3ff58", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2011-03-10T14:14:32-08:00", - "end": "2011-03-10T14:29:32-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "33737001", - "display": "Fracture of rib" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6c5cc660-a1d5-4b7c-b547-afa7c12129e7", - "resource": { - "resourceType": "Claim", - "id": "6c5cc660-a1d5-4b7c-b547-afa7c12129e7", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2", - "resource": { - "resourceType": "Encounter", - "id": "dabf9d6a-0972-4d3e-be29-957f348feec2", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2012-04-19T15:14:32-07:00", - "end": "2012-04-19T15:49:32-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0", - "resource": { - "resourceType": "Condition", - "id": "775e4bf0-7f52-4f37-a03f-e44b9ff11ef0", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" - }, - "dateRecorded": "2012-04-19", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2012-04-19T15:14:32-07:00", - "abatementDateTime": "2012-04-26T15:14:32-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:e3afa29d-f93b-41f7-a0d5-a6712fb519c1", - "resource": { - "resourceType": "Procedure", - "id": "e3afa29d-f93b-41f7-a0d5-a6712fb519c1", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "reasonReference": { - "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" - }, - "performedPeriod": { - "start": "2012-04-19T15:14:32-07:00", - "end": "2012-04-19T15:34:32-07:00" - }, - "encounter": { - "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:43ae4b94-925f-4919-aefc-b0f990a1bc5a", - "resource": { - "resourceType": "MedicationOrder", - "id": "43ae4b94-925f-4919-aefc-b0f990a1bc5a", - "dateWritten": "2012-04-19T15:14:32-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "prescriber": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - }, - "encounter": { - "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" - }, - "reasonReference": { - "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:00f48756-4c62-45ab-8ecf-e30ff789f941", - "resource": { - "resourceType": "Claim", - "id": "00f48756-4c62-45ab-8ecf-e30ff789f941", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:43ae4b94-925f-4919-aefc-b0f990a1bc5a" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3470f76e-8524-4284-ac0a-9c2e9dd1c516", - "resource": { - "resourceType": "CarePlan", - "id": "3470f76e-8524-4284-ac0a-9c2e9dd1c516", - "text": { - "status": "generated", - "div": "
    Respiratory therapy
    " - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "context": { - "reference": "urn:uuid:dabf9d6a-0972-4d3e-be29-957f348feec2" - }, - "period": { - "start": "2012-04-19T15:14:32-07:00", - "end": "2012-04-26T15:14:32-07:00" - }, - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "addresses": [ - { - "reference": "urn:uuid:775e4bf0-7f52-4f37-a03f-e44b9ff11ef0" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "prohibited": false - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "prohibited": false - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:5db17a29-22a6-4a78-b1b9-49293212e6be", - "resource": { - "resourceType": "Claim", - "id": "5db17a29-22a6-4a78-b1b9-49293212e6be", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393", - "resource": { - "resourceType": "Encounter", - "id": "e1ca29b9-6124-4ed0-a617-f7a8a5855393", - "status": "finished", - "class": "outpatient", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2012-04-30T15:14:32-07:00", - "end": "2012-04-30T15:29:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:451e48cb-04fa-44b3-9588-6b7683e3dfa5", - "resource": { - "resourceType": "Observation", - "id": "451e48cb-04fa-44b3-9588-6b7683e3dfa5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb5dd232-761b-4a23-ab5a-92524b7e8576", - "resource": { - "resourceType": "Observation", - "id": "fb5dd232-761b-4a23-ab5a-92524b7e8576", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c76338a-9661-48b1-9243-7ab0bdc4e518", - "resource": { - "resourceType": "Observation", - "id": "7c76338a-9661-48b1-9243-7ab0bdc4e518", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 88.700, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3b3a87c-9dcf-42fc-96f9-70d834112c63", - "resource": { - "resourceType": "Observation", - "id": "b3b3a87c-9dcf-42fc-96f9-70d834112c63", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 28.400, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99f81bd2-b84b-4cb7-bb70-d602599948ee", - "resource": { - "resourceType": "Observation", - "id": "99f81bd2-b84b-4cb7-bb70-d602599948ee", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9df1cdb-9c4a-4684-b579-5cc5b1975fd8", - "resource": { - "resourceType": "Observation", - "id": "a9df1cdb-9c4a-4684-b579-5cc5b1975fd8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 75, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9bc65f53-46e7-43be-813a-e32de96480cd", - "resource": { - "resourceType": "Observation", - "id": "9bc65f53-46e7-43be-813a-e32de96480cd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:06b17005-c565-41fc-9806-3e5db854618b", - "resource": { - "resourceType": "Observation", - "id": "06b17005-c565-41fc-9806-3e5db854618b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 99.760, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8cecb2af-a412-413c-bf53-f7a0f91d19e9", - "resource": { - "resourceType": "Observation", - "id": "8cecb2af-a412-413c-bf53-f7a0f91d19e9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 16.560, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b9026ff5-71ea-47ac-863e-14e600820083", - "resource": { - "resourceType": "Observation", - "id": "b9026ff5-71ea-47ac-863e-14e600820083", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.2200, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e7965cd0-67d8-4a94-8a3c-f8497c1e2459", - "resource": { - "resourceType": "Observation", - "id": "e7965cd0-67d8-4a94-8a3c-f8497c1e2459", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 9.4300, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40a510fc-ffc9-4867-a351-053697bd708f", - "resource": { - "resourceType": "Observation", - "id": "40a510fc-ffc9-4867-a351-053697bd708f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 143.06, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76bf3529-4893-4151-a821-d2c178889f5e", - "resource": { - "resourceType": "Observation", - "id": "76bf3529-4893-4151-a821-d2c178889f5e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.0100, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d74a1e9-e094-42ee-9522-ced256860d49", - "resource": { - "resourceType": "Observation", - "id": "5d74a1e9-e094-42ee-9522-ced256860d49", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 108.82, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:711a3dc5-ceab-44a4-9cdf-2746d702b541", - "resource": { - "resourceType": "Observation", - "id": "711a3dc5-ceab-44a4-9cdf-2746d702b541", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 27.190, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6c938382-dee3-420b-b7e0-f0ccaf952502", - "resource": { - "resourceType": "Observation", - "id": "6c938382-dee3-420b-b7e0-f0ccaf952502", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 169.57, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:03d82c6b-4aef-47d9-b71c-a40c61cfbd1c", - "resource": { - "resourceType": "Observation", - "id": "03d82c6b-4aef-47d9-b71c-a40c61cfbd1c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 107.81, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3af8274a-8f9f-4c02-8f9e-94c6b664acac", - "resource": { - "resourceType": "Observation", - "id": "3af8274a-8f9f-4c02-8f9e-94c6b664acac", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 75.380, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:880d05f0-b73b-4363-8d6e-66b07362793f", - "resource": { - "resourceType": "Observation", - "id": "880d05f0-b73b-4363-8d6e-66b07362793f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 72.640, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69ae792d-5ed5-440e-88b9-b64b53f7f394", - "resource": { - "resourceType": "Observation", - "id": "69ae792d-5ed5-440e-88b9-b64b53f7f394", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:78853a2c-25bc-4ed8-ac19-163887e55b7e", - "resource": { - "resourceType": "Observation", - "id": "78853a2c-25bc-4ed8-ac19-163887e55b7e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.9600, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9761dc4f-9b1e-48cc-ab4a-077b824d9ff1", - "resource": { - "resourceType": "Immunization", - "id": "9761dc4f-9b1e-48cc-ab4a-077b824d9ff1", - "status": "completed", - "date": "2012-04-30T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:feb9d718-6be7-4c32-8e4c-70d69e85a3e5", - "resource": { - "resourceType": "Immunization", - "id": "feb9d718-6be7-4c32-8e4c-70d69e85a3e5", - "status": "completed", - "date": "2012-04-30T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:febb4be4-d5c5-4e72-a1cc-6b913167b2cd", - "resource": { - "resourceType": "Immunization", - "id": "febb4be4-d5c5-4e72-a1cc-6b913167b2cd", - "status": "completed", - "date": "2012-04-30T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - } - ], - "text": "Hep A, adult" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b7e0cf76-da7a-43dd-be2a-16d7645e1449", - "resource": { - "resourceType": "DiagnosticReport", - "id": "b7e0cf76-da7a-43dd-be2a-16d7645e1449", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "result": [ - { - "reference": "urn:uuid:711a3dc5-ceab-44a4-9cdf-2746d702b541", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f2aa89e6-b0b7-427c-b408-5356683187f4", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f2aa89e6-b0b7-427c-b408-5356683187f4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:e1ca29b9-6124-4ed0-a617-f7a8a5855393" - }, - "effectiveDateTime": "2012-04-30T15:14:32-07:00", - "issued": "2012-04-30T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "result": [ - { - "reference": "urn:uuid:880d05f0-b73b-4363-8d6e-66b07362793f", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b3f5fb10-ee9c-4d17-9ad5-ed99460dfcdc", - "resource": { - "resourceType": "Claim", - "id": "b3f5fb10-ee9c-4d17-9ad5-ed99460dfcdc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "52", - "display": "Hep A, adult" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4", - "resource": { - "resourceType": "Encounter", - "id": "873efd7e-4b2d-4ddc-901e-1298549575e4", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2012-06-25T15:14:32-07:00", - "end": "2012-06-25T15:44:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:76acaaeb-feab-45f8-90a5-7cd16e0d0bac", - "resource": { - "resourceType": "Observation", - "id": "76acaaeb-feab-45f8-90a5-7cd16e0d0bac", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e45235d0-d70c-4765-a57a-2d9f7ac15600", - "resource": { - "resourceType": "Observation", - "id": "e45235d0-d70c-4765-a57a-2d9f7ac15600", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:210ae20d-717c-4119-aa79-21e96dd8f549", - "resource": { - "resourceType": "Observation", - "id": "210ae20d-717c-4119-aa79-21e96dd8f549", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 87.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:64b06f8a-773c-4975-a03e-c0e68bdb0797", - "resource": { - "resourceType": "Observation", - "id": "64b06f8a-773c-4975-a03e-c0e68bdb0797", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 28.020, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9eaee75-bb6f-4062-ba6a-29a568e5f558", - "resource": { - "resourceType": "Observation", - "id": "c9eaee75-bb6f-4062-ba6a-29a568e5f558", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 134, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0ab866e-2211-4319-9b7a-9000a7c03282", - "resource": { - "resourceType": "Observation", - "id": "c0ab866e-2211-4319-9b7a-9000a7c03282", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebfe0f30-409d-46f2-b9a2-8b4c6a203a0a", - "resource": { - "resourceType": "Observation", - "id": "ebfe0f30-409d-46f2-b9a2-8b4c6a203a0a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:818d6b41-1c1f-4660-8650-b11a26117115", - "resource": { - "resourceType": "Observation", - "id": "818d6b41-1c1f-4660-8650-b11a26117115", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 66.900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:403d90c0-7d42-4a4e-a6ff-19a45b1bcb1f", - "resource": { - "resourceType": "Observation", - "id": "403d90c0-7d42-4a4e-a6ff-19a45b1bcb1f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 16, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e0a448bd-7986-4c26-8290-43b57c201eef", - "resource": { - "resourceType": "Observation", - "id": "e0a448bd-7986-4c26-8290-43b57c201eef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.1900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31d32f3f-7b1b-4a05-8923-d977e1556af1", - "resource": { - "resourceType": "Observation", - "id": "31d32f3f-7b1b-4a05-8923-d977e1556af1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 10.030, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:739ea4cb-b15e-4461-9291-d037db12946d", - "resource": { - "resourceType": "Observation", - "id": "739ea4cb-b15e-4461-9291-d037db12946d", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 136.54, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a4144224-ca59-490f-b099-5baf1a192e10", - "resource": { - "resourceType": "Observation", - "id": "a4144224-ca59-490f-b099-5baf1a192e10", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 4.4000, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ebcf742-da66-4254-92e6-14f8b70cba67", - "resource": { - "resourceType": "Observation", - "id": "1ebcf742-da66-4254-92e6-14f8b70cba67", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 107.30, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31124929-25dd-4d51-bc45-d22b8553c136", - "resource": { - "resourceType": "Observation", - "id": "31124929-25dd-4d51-bc45-d22b8553c136", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 23.410, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca3ebab8-5c95-4857-b7e2-f337a07cc2e7", - "resource": { - "resourceType": "Observation", - "id": "ca3ebab8-5c95-4857-b7e2-f337a07cc2e7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35d245ee-d6af-4509-a2c4-938f146e2cd3", - "resource": { - "resourceType": "Observation", - "id": "35d245ee-d6af-4509-a2c4-938f146e2cd3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "valueQuantity": { - "value": 6.1000, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:360e5870-25bf-4342-ad22-49cd8e0847c2", - "resource": { - "resourceType": "Procedure", - "id": "360e5870-25bf-4342-ad22-49cd8e0847c2", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2012-06-25T15:14:32-07:00", - "end": "2012-06-25T15:29:32-07:00" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f82d5b51-cadd-4ac0-be01-57b30a8c6b0f", - "resource": { - "resourceType": "Immunization", - "id": "f82d5b51-cadd-4ac0-be01-57b30a8c6b0f", - "status": "completed", - "date": "2012-06-25T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c2c01e9c-0151-4dd3-a500-1ded5efd49a5", - "resource": { - "resourceType": "DiagnosticReport", - "id": "c2c01e9c-0151-4dd3-a500-1ded5efd49a5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:873efd7e-4b2d-4ddc-901e-1298549575e4" - }, - "effectiveDateTime": "2012-06-25T15:14:32-07:00", - "issued": "2012-06-25T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:31124929-25dd-4d51-bc45-d22b8553c136", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b3477957-59a2-4a37-b258-ca2fe6200eba", - "resource": { - "resourceType": "Claim", - "id": "b3477957-59a2-4a37-b258-ca2fe6200eba", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 635.53, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8978465e-2136-41bd-890e-672c8d98f978", - "resource": { - "resourceType": "Encounter", - "id": "8978465e-2136-41bd-890e-672c8d98f978", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2013-02-23T14:14:32-08:00", - "end": "2013-02-23T14:29:32-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1826a772-93d8-4ec1-98c3-1da624bbc0df", - "resource": { - "resourceType": "Condition", - "id": "1826a772-93d8-4ec1-98c3-1da624bbc0df", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:8978465e-2136-41bd-890e-672c8d98f978" - }, - "dateRecorded": "2013-02-23", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2013-02-23T14:14:32-08:00", - "abatementDateTime": "2013-03-09T14:14:32-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ce88550b-de77-4e2d-a157-8698243a5dfc", - "resource": { - "resourceType": "Claim", - "id": "ce88550b-de77-4e2d-a157-8698243a5dfc", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641", - "resource": { - "resourceType": "Encounter", - "id": "eed22565-60eb-4729-b7f5-896802b09641", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2014-06-30T15:14:32-07:00", - "end": "2014-06-30T15:44:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4fd1696c-fa99-43b8-ad3b-3abbf234a614", - "resource": { - "resourceType": "Observation", - "id": "4fd1696c-fa99-43b8-ad3b-3abbf234a614", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d42f456-ecce-4e9e-ad34-6d20db8ab120", - "resource": { - "resourceType": "Observation", - "id": "5d42f456-ecce-4e9e-ad34-6d20db8ab120", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e19bf84d-0689-49e1-a5d8-ba51d76aa8d1", - "resource": { - "resourceType": "Observation", - "id": "e19bf84d-0689-49e1-a5d8-ba51d76aa8d1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 90, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:afcf176a-69ed-4256-9bca-1ce65590731e", - "resource": { - "resourceType": "Observation", - "id": "afcf176a-69ed-4256-9bca-1ce65590731e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 28.840, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd5f84ee-3b40-4932-a91b-6b13b2e24bb1", - "resource": { - "resourceType": "Observation", - "id": "dd5f84ee-3b40-4932-a91b-6b13b2e24bb1", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 125, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e36207d5-1342-485b-84db-d0cecd97910a", - "resource": { - "resourceType": "Observation", - "id": "e36207d5-1342-485b-84db-d0cecd97910a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 100, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9acd6700-cf70-498b-9bae-7465c87dd32f", - "resource": { - "resourceType": "Observation", - "id": "9acd6700-cf70-498b-9bae-7465c87dd32f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:455fdc2c-e12f-4049-b627-1040fa5b3aab", - "resource": { - "resourceType": "Observation", - "id": "455fdc2c-e12f-4049-b627-1040fa5b3aab", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 93.390, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3044abe1-dde8-4b49-bc17-c810c9835341", - "resource": { - "resourceType": "Observation", - "id": "3044abe1-dde8-4b49-bc17-c810c9835341", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 8.6600, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5b84e319-3b02-4297-b414-ee65dcb906a5", - "resource": { - "resourceType": "Observation", - "id": "5b84e319-3b02-4297-b414-ee65dcb906a5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.2000, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c27010f-ad80-451d-a0de-ffacf4ceb5fd", - "resource": { - "resourceType": "Observation", - "id": "3c27010f-ad80-451d-a0de-ffacf4ceb5fd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 9.4500, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9cdd64a8-0ceb-49ae-8bfd-5036703c2b67", - "resource": { - "resourceType": "Observation", - "id": "9cdd64a8-0ceb-49ae-8bfd-5036703c2b67", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 143.41, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c177516-908e-449e-9754-0df337a5f275", - "resource": { - "resourceType": "Observation", - "id": "2c177516-908e-449e-9754-0df337a5f275", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 3.7900, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5dd27b3d-397e-455e-beab-956ad546db04", - "resource": { - "resourceType": "Observation", - "id": "5dd27b3d-397e-455e-beab-956ad546db04", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 108.61, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ab0dd46-8cd8-4290-8e96-76d6f26036a4", - "resource": { - "resourceType": "Observation", - "id": "2ab0dd46-8cd8-4290-8e96-76d6f26036a4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 22.180, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b5da3c37-0527-415c-aaf5-894e419a6f26", - "resource": { - "resourceType": "Observation", - "id": "b5da3c37-0527-415c-aaf5-894e419a6f26", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b33a4c9-50fe-4599-ab1a-b333f07810f7", - "resource": { - "resourceType": "Observation", - "id": "3b33a4c9-50fe-4599-ab1a-b333f07810f7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.9500, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83fdf235-73f5-4484-acbb-61c4a800055e", - "resource": { - "resourceType": "Procedure", - "id": "83fdf235-73f5-4484-acbb-61c4a800055e", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "performedPeriod": { - "start": "2014-06-30T15:14:32-07:00", - "end": "2014-06-30T15:29:32-07:00" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f9c1747b-4233-4254-b600-4936fc774839", - "resource": { - "resourceType": "Immunization", - "id": "f9c1747b-4233-4254-b600-4936fc774839", - "status": "completed", - "date": "2014-06-30T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:61355308-bb5e-4491-8b5b-a872c0ec30fa", - "resource": { - "resourceType": "DiagnosticReport", - "id": "61355308-bb5e-4491-8b5b-a872c0ec30fa", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:eed22565-60eb-4729-b7f5-896802b09641" - }, - "effectiveDateTime": "2014-06-30T15:14:32-07:00", - "issued": "2014-06-30T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:2ab0dd46-8cd8-4290-8e96-76d6f26036a4", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:fdd528a6-5d63-4798-8107-9b57be4c82b4", - "resource": { - "resourceType": "Claim", - "id": "fdd528a6-5d63-4798-8107-9b57be4c82b4", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 490.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481", - "resource": { - "resourceType": "Encounter", - "id": "7482b173-fb3c-4239-8d43-082d6ed7c481", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2016-07-04T15:14:32-07:00", - "end": "2016-07-04T15:29:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8bc0126f-d281-43c0-ab46-eb5319dfa430", - "resource": { - "resourceType": "Observation", - "id": "8bc0126f-d281-43c0-ab46-eb5319dfa430", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:433113f8-9e45-444b-b6af-9ce64ab732e9", - "resource": { - "resourceType": "Observation", - "id": "433113f8-9e45-444b-b6af-9ce64ab732e9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ccf232f-da8a-472f-9b26-ed1cc7a991f0", - "resource": { - "resourceType": "Observation", - "id": "1ccf232f-da8a-472f-9b26-ed1cc7a991f0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 93.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7cfb768-6973-4722-bae6-51b6b4bcd4f4", - "resource": { - "resourceType": "Observation", - "id": "a7cfb768-6973-4722-bae6-51b6b4bcd4f4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 30.070, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:44f6ae8b-a2b1-4e9f-aaaf-8991a7c51451", - "resource": { - "resourceType": "Observation", - "id": "44f6ae8b-a2b1-4e9f-aaaf-8991a7c51451", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 125, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:24f44567-c7df-463c-8288-c0c2ed6efa88", - "resource": { - "resourceType": "Observation", - "id": "24f44567-c7df-463c-8288-c0c2ed6efa88", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 76, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9d4a2ab-0bf8-457d-89aa-9cf0b6716ef0", - "resource": { - "resourceType": "Observation", - "id": "a9d4a2ab-0bf8-457d-89aa-9cf0b6716ef0", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2c4624b-9709-464d-8efd-c837d247abfd", - "resource": { - "resourceType": "Observation", - "id": "a2c4624b-9709-464d-8efd-c837d247abfd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 65.420, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bdb853cb-31bf-44e5-8f2d-543f3afb7c21", - "resource": { - "resourceType": "Observation", - "id": "bdb853cb-31bf-44e5-8f2d-543f3afb7c21", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 11.310, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:009b42e2-42d3-4ae8-a3ba-1983f7841d26", - "resource": { - "resourceType": "Observation", - "id": "009b42e2-42d3-4ae8-a3ba-1983f7841d26", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.2300, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f91abd1d-f0a8-4fda-ac43-eecd7800efd4", - "resource": { - "resourceType": "Observation", - "id": "f91abd1d-f0a8-4fda-ac43-eecd7800efd4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 9.1500, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e39ea985-1c8d-4aca-b4ec-fbb58b2ef101", - "resource": { - "resourceType": "Observation", - "id": "e39ea985-1c8d-4aca-b4ec-fbb58b2ef101", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 142.53, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5fcdae25-f68b-46af-b8fb-851f673386e9", - "resource": { - "resourceType": "Observation", - "id": "5fcdae25-f68b-46af-b8fb-851f673386e9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 3.9800, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c8b2cbb-db18-4ec7-9800-cfa7a5973274", - "resource": { - "resourceType": "Observation", - "id": "2c8b2cbb-db18-4ec7-9800-cfa7a5973274", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 106.89, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", - "resource": { - "resourceType": "Observation", - "id": "2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 20.660, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1718324-d11b-4aee-aa35-5c40c4921350", - "resource": { - "resourceType": "Observation", - "id": "d1718324-d11b-4aee-aa35-5c40c4921350", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 174.91, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5aacdec5-80e3-4563-86ef-fefaaa6ba9fd", - "resource": { - "resourceType": "Observation", - "id": "5aacdec5-80e3-4563-86ef-fefaaa6ba9fd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 101.86, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c1acfc1e-e102-4d2b-884d-8fd6c51b6e85", - "resource": { - "resourceType": "Observation", - "id": "c1acfc1e-e102-4d2b-884d-8fd6c51b6e85", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 78.310, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84bec837-ecf8-471d-a1f3-b51fe28263c9", - "resource": { - "resourceType": "Observation", - "id": "84bec837-ecf8-471d-a1f3-b51fe28263c9", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 76.220, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e76e0dbf-8992-4794-a692-628902a9530a", - "resource": { - "resourceType": "Observation", - "id": "e76e0dbf-8992-4794-a692-628902a9530a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 9.3247, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc9bd4d1-cec0-4f99-8fa0-632c06214642", - "resource": { - "resourceType": "Observation", - "id": "fc9bd4d1-cec0-4f99-8fa0-632c06214642", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.3298, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:219782f8-d876-46eb-86bc-8d5fc2f6a422", - "resource": { - "resourceType": "Observation", - "id": "219782f8-d876-46eb-86bc-8d5fc2f6a422", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 15.912, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f82ebde3-5dd8-4cde-8fcd-119d72c33e83", - "resource": { - "resourceType": "Observation", - "id": "f82ebde3-5dd8-4cde-8fcd-119d72c33e83", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 46.435, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d5e915c-64e7-4e74-ae43-d80bd7110f96", - "resource": { - "resourceType": "Observation", - "id": "7d5e915c-64e7-4e74-ae43-d80bd7110f96", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 83.143, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:13c324d9-3631-465c-adf9-3b48ccf0bc21", - "resource": { - "resourceType": "Observation", - "id": "13c324d9-3631-465c-adf9-3b48ccf0bc21", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 32.845, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68cc0ac4-786c-4da2-9f30-8b976685565a", - "resource": { - "resourceType": "Observation", - "id": "68cc0ac4-786c-4da2-9f30-8b976685565a", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 34.571, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ffb9a032-a068-49cb-b466-00d73f195ca8", - "resource": { - "resourceType": "Observation", - "id": "ffb9a032-a068-49cb-b466-00d73f195ca8", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 43.802, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e04d93f-524c-48b5-b4f7-14fe8e27506f", - "resource": { - "resourceType": "Observation", - "id": "3e04d93f-524c-48b5-b4f7-14fe8e27506f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 381.72, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0530c7db-fa1a-4dee-86bb-5830b0b53edd", - "resource": { - "resourceType": "Observation", - "id": "0530c7db-fa1a-4dee-86bb-5830b0b53edd", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 396.29, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30bf7ea3-6c8d-428a-8cc6-20439d26b69e", - "resource": { - "resourceType": "Observation", - "id": "30bf7ea3-6c8d-428a-8cc6-20439d26b69e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 9.6093, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0fc3a1d9-ad7b-42bf-948f-e88c451135ef", - "resource": { - "resourceType": "Observation", - "id": "0fc3a1d9-ad7b-42bf-948f-e88c451135ef", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2afe1792-3e43-4b67-9cf9-2e47110bb8a5", - "resource": { - "resourceType": "Observation", - "id": "2afe1792-3e43-4b67-9cf9-2e47110bb8a5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "valueQuantity": { - "value": 6.1200, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e96ae61-06f9-4a98-9617-5adca8e69d25", - "resource": { - "resourceType": "Immunization", - "id": "8e96ae61-06f9-4a98-9617-5adca8e69d25", - "status": "completed", - "date": "2016-07-04T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0fb40172-97c2-4e0e-b42a-56d9c05e4be4", - "resource": { - "resourceType": "DiagnosticReport", - "id": "0fb40172-97c2-4e0e-b42a-56d9c05e4be4", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:2cf2a08b-5231-48ef-98d2-599dbfbd1cc7", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:e5997cee-7e27-477b-b525-4da465317220", - "resource": { - "resourceType": "DiagnosticReport", - "id": "e5997cee-7e27-477b-b525-4da465317220", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:84bec837-ecf8-471d-a1f3-b51fe28263c9", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:3adba72e-d38c-4a9b-be2a-e860e4ad9a4e", - "resource": { - "resourceType": "DiagnosticReport", - "id": "3adba72e-d38c-4a9b-be2a-e860e4ad9a4e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:7482b173-fb3c-4239-8d43-082d6ed7c481" - }, - "effectiveDateTime": "2016-07-04T15:14:32-07:00", - "issued": "2016-07-04T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:30bf7ea3-6c8d-428a-8cc6-20439d26b69e", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:280cae6e-d63f-41ad-83ec-d858a3c4044d", - "resource": { - "resourceType": "Claim", - "id": "280cae6e-d63f-41ad-83ec-d858a3c4044d", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a", - "resource": { - "resourceType": "Encounter", - "id": "ae251a8e-616a-4ada-9296-217127bb490a", - "status": "finished", - "class": "ambulatory", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - } - } - ], - "period": { - "start": "2016-10-09T15:14:32-07:00", - "end": "2016-10-09T15:44:32-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd", - "resource": { - "resourceType": "Condition", - "id": "1b764010-ac5a-491e-8ec5-16bab53cd5bd", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" - }, - "dateRecorded": "2016-10-09", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - }, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/condition-category", - "code": "diagnosis" - } - ] - }, - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "onsetDateTime": "2016-10-09T15:14:32-07:00", - "abatementDateTime": "2016-10-18T15:14:32-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:3774a010-5e60-45f5-8d76-a6bfda4b715c", - "resource": { - "resourceType": "Observation", - "id": "3774a010-5e60-45f5-8d76-a6bfda4b715c", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" - }, - "effectiveDateTime": "2016-10-09T15:14:32-07:00", - "issued": "2016-10-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 38.005, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cdfb1614-c8df-48a0-be98-75a0f2caf468", - "resource": { - "resourceType": "Procedure", - "id": "cdfb1614-c8df-48a0-be98-75a0f2caf468", - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "reasonReference": { - "reference": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd" - }, - "performedPeriod": { - "start": "2016-10-09T15:14:32-07:00", - "end": "2016-10-09T15:29:32-07:00" - }, - "encounter": { - "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:1f5f0725-6902-4150-a0df-ed7c23553d5d", - "resource": { - "resourceType": "MedicationOrder", - "id": "1f5f0725-6902-4150-a0df-ed7c23553d5d", - "dateWritten": "2016-10-09T15:14:32-07:00", - "status": "stopped", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "prescriber": { - "reference": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7" - }, - "encounter": { - "reference": "urn:uuid:ae251a8e-616a-4ada-9296-217127bb490a" - }, - "reasonReference": { - "reference": "urn:uuid:1b764010-ac5a-491e-8ec5-16bab53cd5bd" - }, - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "834102", - "display": "Penicillin V Potassium 500 MG Oral Tablet" - } - ], - "text": "Penicillin V Potassium 500 MG Oral Tablet" - } - }, - "request": { - "method": "POST", - "url": "MedicationOrder" - } - }, - { - "fullUrl": "urn:uuid:187c7568-54cf-456e-a022-3699bd3d7a67", - "resource": { - "resourceType": "Claim", - "id": "187c7568-54cf-456e-a022-3699bd3d7a67", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "prescription": { - "reference": "urn:uuid:1f5f0725-6902-4150-a0df-ed7c23553d5d" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f4fcb9e1-936f-4fec-9a36-0898866f3e6b", - "resource": { - "resourceType": "Claim", - "id": "f4fcb9e1-936f-4fec-9a36-0898866f3e6b", - "type": "institutional", - "organization": { - "reference": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12" - }, - "use": "complete", - "diagnosis": [ - { - "sequence": 1, - "diagnosis": { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - }, - { - "sequence": 3, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - }, - "net": { - "value": 2325.12, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f", - "resource": { - "resourceType": "Encounter", - "id": "9029c1c3-3e90-430f-84d3-2710961f1a5f", - "status": "finished", - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd" - } - } - ], - "period": { - "start": "2018-07-09T15:14:32-07:00", - "end": "2018-07-09T15:29:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d6208ddf-79f4-4d9b-bac6-96639bc25963", - "resource": { - "resourceType": "Observation", - "id": "d6208ddf-79f4-4d9b-bac6-96639bc25963", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 176.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e92feaa-b6f5-49ef-bbbb-a693a8643268", - "resource": { - "resourceType": "Observation", - "id": "3e92feaa-b6f5-49ef-bbbb-a693a8643268", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a959df95-0bf6-44c7-86f1-0a68bc276de3", - "resource": { - "resourceType": "Observation", - "id": "a959df95-0bf6-44c7-86f1-0a68bc276de3", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 88, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8562e56f-eab7-4621-a02a-763615fb9e89", - "resource": { - "resourceType": "Observation", - "id": "8562e56f-eab7-4621-a02a-763615fb9e89", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 28.190, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:39a96f7d-d019-4f53-9e0d-c4ffc0d6eecf", - "resource": { - "resourceType": "Observation", - "id": "39a96f7d-d019-4f53-9e0d-c4ffc0d6eecf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 72, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a691721a-19b9-44ad-a0bb-951e2757e629", - "resource": { - "resourceType": "Observation", - "id": "a691721a-19b9-44ad-a0bb-951e2757e629", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4d392dee-f593-4de4-aabe-6fdecd5c055f", - "resource": { - "resourceType": "Observation", - "id": "4d392dee-f593-4de4-aabe-6fdecd5c055f", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ], - "text": "vital-signs" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5cd46b09-84f7-4a7b-a0d7-5c1a8b9b1edf", - "resource": { - "resourceType": "Observation", - "id": "5cd46b09-84f7-4a7b-a0d7-5c1a8b9b1edf", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 88.960, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d659949-e96b-41ab-b4bc-7ef6d45c296b", - "resource": { - "resourceType": "Observation", - "id": "0d659949-e96b-41ab-b4bc-7ef6d45c296b", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 19.420, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cec03bed-4729-4d42-8010-467d80929cec", - "resource": { - "resourceType": "Observation", - "id": "cec03bed-4729-4d42-8010-467d80929cec", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 1.1200, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:695c8154-110e-462a-ab0d-f536809ed44e", - "resource": { - "resourceType": "Observation", - "id": "695c8154-110e-462a-ab0d-f536809ed44e", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 8.6900, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25dca154-f59e-4a62-ab3d-5f0b0e8fc915", - "resource": { - "resourceType": "Observation", - "id": "25dca154-f59e-4a62-ab3d-5f0b0e8fc915", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 141.33, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:215a790f-9080-4fb9-b950-35ba6fc36a41", - "resource": { - "resourceType": "Observation", - "id": "215a790f-9080-4fb9-b950-35ba6fc36a41", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.1800, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:17248f47-1630-4e44-9d77-122d8cb37da5", - "resource": { - "resourceType": "Observation", - "id": "17248f47-1630-4e44-9d77-122d8cb37da5", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 109.62, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f4fb0f1c-0600-4912-8cb3-ca4091597c49", - "resource": { - "resourceType": "Observation", - "id": "f4fb0f1c-0600-4912-8cb3-ca4091597c49", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 27.860, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:562fc911-423f-46f4-ae18-c56071403825", - "resource": { - "resourceType": "Observation", - "id": "562fc911-423f-46f4-ae18-c56071403825", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ], - "text": "survey" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:948cc9ea-644e-4430-aa6b-bce7df6983dc", - "resource": { - "resourceType": "Observation", - "id": "948cc9ea-644e-4430-aa6b-bce7df6983dc", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ], - "text": "laboratory" - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "valueQuantity": { - "value": 5.9600, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba7baf02-d772-4d35-9308-cdbe2c9a3190", - "resource": { - "resourceType": "Immunization", - "id": "ba7baf02-d772-4d35-9308-cdbe2c9a3190", - "status": "completed", - "date": "2018-07-09T15:14:32-07:00", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "wasNotGiven": false, - "reported": false, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - } - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:badb5f6a-073f-4ad6-b0c2-4a733cc40c89", - "resource": { - "resourceType": "DiagnosticReport", - "id": "badb5f6a-073f-4ad6-b0c2-4a733cc40c89", - "status": "final", - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/ValueSet/diagnostic-service-sections", - "code": "LAB" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "encounter": { - "reference": "urn:uuid:9029c1c3-3e90-430f-84d3-2710961f1a5f" - }, - "effectiveDateTime": "2018-07-09T15:14:32-07:00", - "issued": "2018-07-09T15:14:32.480-07:00", - "performer": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "result": [ - { - "reference": "urn:uuid:f4fb0f1c-0600-4912-8cb3-ca4091597c49", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:6c348dfd-8fe1-4f8e-95f7-6e30e73369ee", - "resource": { - "resourceType": "Claim", - "id": "6c348dfd-8fe1-4f8e-95f7-6e30e73369ee", - "type": "institutional", - "organization": { - "reference": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - }, - "use": "complete", - "patient": { - "reference": "urn:uuid:f04ef974-6d6f-4e1c-804f-cd4d62aabb4f" - }, - "item": [ - { - "sequence": 1, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - }, - { - "sequence": 2, - "type": { - "system": "http://hl7.org/fhir/v3/ActCode", - "code": "CSINV", - "display": "clinical service invoice" - }, - "service": { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Claim" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json deleted file mode 100644 index 4b9fdcc72d56..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/hospitalInformation1586309771387.json +++ /dev/null @@ -1,825 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:49318f80-bd8b-3fc7-a096-ac43088b0c12", - "resource": { - "resourceType": "Organization", - "id": "49318f80-bd8b-3fc7-a096-ac43088b0c12", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 21 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "49318f80-bd8b-3fc7-a096-ac43088b0c12" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "COOLEY DICKINSON HOSPITAL INC,THE", - "telecom": [ - { - "system": "phone", - "value": "4135822000" - } - ], - "address": [ - { - "line": [ - "30 LOCUST STREET" - ], - "city": "NORTHAMPTON", - "state": "MA", - "postalCode": "01060", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:ecc51621-0af3-3b35-ac3e-8b1e34022e92", - "resource": { - "resourceType": "Organization", - "id": "ecc51621-0af3-3b35-ac3e-8b1e34022e92", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 145 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ecc51621-0af3-3b35-ac3e-8b1e34022e92" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "SAINT ANNE'S HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5086745600" - } - ], - "address": [ - { - "line": [ - "795 MIDDLE STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02721", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:08bcda9c-f8c8-3244-82d4-fc306a7a55d3", - "resource": { - "resourceType": "Organization", - "id": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "08bcda9c-f8c8-3244-82d4-fc306a7a55d3" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "BOSTON MEDICAL CENTER CORPORATION-", - "telecom": [ - { - "system": "phone", - "value": "6176388000" - } - ], - "address": [ - { - "line": [ - "1 BOSTON MEDICAL CENTER PLACE" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02118", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:4861d01f-019c-3dac-a153-8334e50919f9", - "resource": { - "resourceType": "Organization", - "id": "4861d01f-019c-3dac-a153-8334e50919f9", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 119 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "4861d01f-019c-3dac-a153-8334e50919f9" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "NORTH SHORE MEDICAL CENTER -", - "telecom": [ - { - "system": "phone", - "value": "9787411215" - } - ], - "address": [ - { - "line": [ - "81 HIGHLAND AVENUE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f4e7709c-02f6-37ca-aeea-8247d74e88e7", - "resource": { - "resourceType": "Organization", - "id": "f4e7709c-02f6-37ca-aeea-8247d74e88e7", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 113 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f4e7709c-02f6-37ca-aeea-8247d74e88e7" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "WINCHESTER HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "7817299000" - } - ], - "address": [ - { - "line": [ - "41 HIGHLAND AVENUE" - ], - "city": "WINCHESTER", - "state": "MA", - "postalCode": "01890", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:b7175ab4-bde5-3848-891b-579bccb77c7c", - "resource": { - "resourceType": "Organization", - "id": "b7175ab4-bde5-3848-891b-579bccb77c7c", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "b7175ab4-bde5-3848-891b-579bccb77c7c" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "TUFTS MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "6176365000" - } - ], - "address": [ - { - "line": [ - "800 WASHINGTON STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02111", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", - "resource": { - "resourceType": "Organization", - "id": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 39 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 32 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 22 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 12 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "1f5f3cb7-ad8d-3c50-a15d-470d2a723b84" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "PCP15859", - "telecom": [ - { - "system": "phone", - "value": "413-387-2118" - } - ], - "address": [ - { - "line": [ - "92 MAIN ST" - ], - "city": "FLORENCE", - "state": "MA", - "postalCode": "01062-1460", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:245f252c-be87-3017-8a0b-a04448a97219", - "resource": { - "resourceType": "Organization", - "id": "245f252c-be87-3017-8a0b-a04448a97219", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 48 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 264 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 19 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 27 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "245f252c-be87-3017-8a0b-a04448a97219" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "PCP45531", - "telecom": [ - { - "system": "phone", - "value": "617-230-9940" - } - ], - "address": [ - { - "line": [ - "661 MASSACHUSETTS AVE" - ], - "city": "ARLINGTON", - "state": "MA", - "postalCode": "02476-5001", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f9914571-32b8-36d3-a422-4cdce9fbb86c", - "resource": { - "resourceType": "Organization", - "id": "f9914571-32b8-36d3-a422-4cdce9fbb86c", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 56 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 415 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 45 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 27 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f9914571-32b8-36d3-a422-4cdce9fbb86c" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "INDEPENDENCE EYE ASSOCIATES, PC", - "telecom": [ - { - "system": "phone", - "value": "508-985-6600" - } - ], - "address": [ - { - "line": [ - "365 FAUNCE CORNER RD" - ], - "city": "DARTMOUTH", - "state": "MA", - "postalCode": "02747-6230", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:70d8b93b-cc76-3c8b-8929-5aea213ecc86", - "resource": { - "resourceType": "Organization", - "id": "70d8b93b-cc76-3c8b-8929-5aea213ecc86", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 18 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 11 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 113 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "70d8b93b-cc76-3c8b-8929-5aea213ecc86" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "BROOKLINE DERMATOLOGY ASSOCIATES, PC", - "telecom": [ - { - "system": "phone", - "value": "617-608-1575" - } - ], - "address": [ - { - "line": [ - "1208 B VFW PKWY" - ], - "city": "WEST ROXBURY", - "state": "MA", - "postalCode": "02132-4350", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:cab5722d-8856-3fc3-9a04-d3fe34d3e41e", - "resource": { - "resourceType": "Organization", - "id": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 37 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 280 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 13 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 22 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "cab5722d-8856-3fc3-9a04-d3fe34d3e41e" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "COMMONWEALTH RADIOLOGY ASSOCIATES, INC", - "telecom": [ - { - "system": "phone", - "value": "978-741-1200" - } - ], - "address": [ - { - "line": [ - "81 HIGHLAND AVE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970-2714", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:a76bda1d-f42d-3046-b8c6-021f3db0a086", - "resource": { - "resourceType": "Organization", - "id": "a76bda1d-f42d-3046-b8c6-021f3db0a086", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "a76bda1d-f42d-3046-b8c6-021f3db0a086" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "WALK IN AND PRIMARY CARE", - "telecom": [ - { - "system": "phone", - "value": "781-894-8200" - } - ], - "address": [ - { - "line": [ - "808 MAIN STREET" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2451", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f7a6cc79-3075-3124-9f62-6fe562440b1d", - "resource": { - "resourceType": "Organization", - "id": "f7a6cc79-3075-3124-9f62-6fe562440b1d", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "f7a6cc79-3075-3124-9f62-6fe562440b1d" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "HAWTHORN MEDICAL URGENT CARE CENTER", - "telecom": [ - { - "system": "phone", - "value": "508-961-0861" - } - ], - "address": [ - { - "line": [ - "237 STATE ROAD" - ], - "city": "NORTH DARTMOUTH", - "state": "MA", - "postalCode": "2747", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json b/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json deleted file mode 100644 index 59d8bb2107c0..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/DSTU2/practitionerInformation1586309771387.json +++ /dev/null @@ -1,617 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:c1a3f738-c767-30ab-951c-5c60f3c569c7", - "resource": { - "resourceType": "Practitioner", - "id": "c1a3f738-c767-30ab-951c-5c60f3c569c7", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 21 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "60" - } - ], - "active": true, - "name": { - "family": [ - "Hermiston71" - ], - "given": [ - "Andrea7" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "30 LOCUST STREET" - ], - "city": "NORTHAMPTON", - "state": "MA", - "postalCode": "01060", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:38a05f68-cc3b-33b6-993e-1d2bb2696c3d", - "resource": { - "resourceType": "Practitioner", - "id": "38a05f68-cc3b-33b6-993e-1d2bb2696c3d", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 145 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "100" - } - ], - "active": true, - "name": { - "family": [ - "Padberg411" - ], - "given": [ - "Ligia986" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "795 MIDDLE STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02721", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", - "resource": { - "resourceType": "Practitioner", - "id": "64d6ff1b-ef19-3680-80ea-aa6bcbc550bd", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "140" - } - ], - "active": true, - "name": { - "family": [ - "Kuvalis369" - ], - "given": [ - "Maricruz991" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "1 BOSTON MEDICAL CENTER PLACE" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02118", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:b102f703-9e8c-37a2-8eea-08b67c680a1e", - "resource": { - "resourceType": "Practitioner", - "id": "b102f703-9e8c-37a2-8eea-08b67c680a1e", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 119 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "160" - } - ], - "active": true, - "name": { - "family": [ - "Roob72" - ], - "given": [ - "Isobel140" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "81 HIGHLAND AVENUE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:a0153fac-7137-30d8-bb5a-0cb7af968203", - "resource": { - "resourceType": "Practitioner", - "id": "a0153fac-7137-30d8-bb5a-0cb7af968203", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 113 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "430" - } - ], - "active": true, - "name": { - "family": [ - "Carroll471" - ], - "given": [ - "Emmitt44" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "41 HIGHLAND AVENUE" - ], - "city": "WINCHESTER", - "state": "MA", - "postalCode": "01890", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:002862dc-5ff1-380e-82ad-a3cf9c436044", - "resource": { - "resourceType": "Practitioner", - "id": "002862dc-5ff1-380e-82ad-a3cf9c436044", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "470" - } - ], - "active": true, - "name": { - "family": [ - "Orn563" - ], - "given": [ - "Serina556" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "800 WASHINGTON STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02111", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:6d5162ad-3294-398e-9eb4-7021465d4bcd", - "resource": { - "resourceType": "Practitioner", - "id": "6d5162ad-3294-398e-9eb4-7021465d4bcd", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 39 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10930" - } - ], - "active": true, - "name": { - "family": [ - "Pfeffer420" - ], - "given": [ - "Darryl392" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "92 MAIN ST" - ], - "city": "FLORENCE", - "state": "MA", - "postalCode": "01062-1460", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:8e9a1427-af53-3468-9d67-db3b9191c240", - "resource": { - "resourceType": "Practitioner", - "id": "8e9a1427-af53-3468-9d67-db3b9191c240", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 48 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "28020" - } - ], - "active": true, - "name": { - "family": [ - "Waters156" - ], - "given": [ - "Damien170" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "661 MASSACHUSETTS AVE" - ], - "city": "ARLINGTON", - "state": "MA", - "postalCode": "02476-5001", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:7e727355-643f-3e2b-b89f-f5076660354b", - "resource": { - "resourceType": "Practitioner", - "id": "7e727355-643f-3e2b-b89f-f5076660354b", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 56 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "61860" - } - ], - "active": true, - "name": { - "family": [ - "Salazar800" - ], - "given": [ - "Rodrigo242" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "365 FAUNCE CORNER RD" - ], - "city": "DARTMOUTH", - "state": "MA", - "postalCode": "02747-6230", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:2d307854-d23a-337c-8215-e37f84fdc26e", - "resource": { - "resourceType": "Practitioner", - "id": "2d307854-d23a-337c-8215-e37f84fdc26e", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 18 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "66350" - } - ], - "active": true, - "name": { - "family": [ - "Eichmann909" - ], - "given": [ - "Marylou497" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "1208 B VFW PKWY" - ], - "city": "WEST ROXBURY", - "state": "MA", - "postalCode": "02132-4350", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:f4f50d8e-b084-3bc2-a787-30f3a29983fc", - "resource": { - "resourceType": "Practitioner", - "id": "f4f50d8e-b084-3bc2-a787-30f3a29983fc", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 37 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "78430" - } - ], - "active": true, - "name": { - "family": [ - "Keeling57" - ], - "given": [ - "Johana303" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "81 HIGHLAND AVE" - ], - "city": "SALEM", - "state": "MA", - "postalCode": "01970-2714", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:2b592db4-9f8b-3284-bac5-60dc80ccce4c", - "resource": { - "resourceType": "Practitioner", - "id": "2b592db4-9f8b-3284-bac5-60dc80ccce4c", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93270" - } - ], - "active": true, - "name": { - "family": [ - "Padberg411" - ], - "given": [ - "Ligia986" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "808 MAIN STREET" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2451", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:d4b02c0c-6cc4-31dd-bc8a-817f786b043a", - "resource": { - "resourceType": "Practitioner", - "id": "d4b02c0c-6cc4-31dd-bc8a-817f786b043a", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93530" - } - ], - "active": true, - "name": { - "family": [ - "Doyle959" - ], - "given": [ - "Willis868" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "237 STATE ROAD" - ], - "city": "NORTH DARTMOUTH", - "state": "MA", - "postalCode": "2747", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json deleted file mode 100644 index 3bedbb9b820d..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Emerson869_Prohaska837_d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6.json +++ /dev/null @@ -1,16391 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "resource": { - "resourceType": "Patient", - "id": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -6045139225077477987 Population seed: 1586368870505
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Pura348 Grant908" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", - "valueAddress": { - "city": "Boston", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 1.3903699305216464 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 50.60963006947836 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "SS", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-80-1133" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99981444" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "PPN", - "display": "Passport Number" - } - ], - "text": "Passport Number" - }, - "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", - "value": "X32551053X" - } - ], - "name": [ - { - "use": "official", - "family": "Prohaska837", - "given": [ - "Emerson869" - ], - "prefix": [ - "Mr." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-300-9464", - "use": "home" - } - ], - "gender": "male", - "birthDate": "1968-01-01", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.582871446661336 - }, - { - "url": "longitude", - "valueDecimal": -71.29215663584894 - } - ] - } - ], - "line": [ - "113 Toy Mews Unit 67" - ], - "city": "Billerica", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", - "code": "M", - "display": "M" - } - ], - "text": "M" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "resource": { - "resourceType": "Organization", - "id": "b0e04623-b02c-3f8b-92ea-943fc4db60da", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "b0e04623-b02c-3f8b-92ea-943fc4db60da" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "LOWELL GENERAL HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "9789376000" - } - ], - "address": [ - { - "line": [ - "295 VARNUM AVENUE" - ], - "city": "LOWELL", - "state": "MA", - "postalCode": "01854", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "resource": { - "resourceType": "Practitioner", - "id": "57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999759" - } - ], - "active": true, - "name": [ - { - "family": "Tillman293", - "given": [ - "Franklin857" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Franklin857.Tillman293@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "295 VARNUM AVENUE" - ], - "city": "LOWELL", - "state": "MA", - "postalCode": "01854", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948", - "resource": { - "resourceType": "Encounter", - "id": "a9cda84f-edc5-442b-abb6-45625823f948", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "individual": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - } - ], - "period": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "446096008", - "display": "Perennial allergic rhinitis" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d", - "resource": { - "resourceType": "Condition", - "id": "7d6bd517-ca1c-4418-8d83-db5eb534a11d", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "446096008", - "display": "Perennial allergic rhinitis" - } - ], - "text": "Perennial allergic rhinitis" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" - }, - "onsetDateTime": "1972-01-07T04:21:42-08:00", - "recordedDate": "1972-01-07T04:21:42-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:6e71bd2a-e1e4-428b-ad68-f13dea7024ac", - "resource": { - "resourceType": "MedicationRequest", - "id": "6e71bd2a-e1e4-428b-ad68-f13dea7024ac", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "477045", - "display": "Chlorpheniramine Maleate 2 MG/ML Oral Solution" - } - ], - "text": "Chlorpheniramine Maleate 2 MG/ML Oral Solution" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" - }, - "authoredOn": "1972-01-21T04:21:42-08:00", - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:196aa54f-3941-49dd-a683-c97cce2fbd7f", - "resource": { - "resourceType": "Claim", - "id": "196aa54f-3941-49dd-a683-c97cce2fbd7f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "created": "1972-01-21T04:21:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:6e71bd2a-e1e4-428b-ad68-f13dea7024ac" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" - } - ] - } - ], - "total": { - "value": 6.37, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:683826c9-9958-41bb-8068-db0949431668", - "resource": { - "resourceType": "Claim", - "id": "683826c9-9958-41bb-8068-db0949431668", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "created": "1972-01-21T04:21:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "446096008", - "display": "Perennial allergic rhinitis" - } - ], - "text": "Perennial allergic rhinitis" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:55eaebcc-0bb8-46e5-bb2b-7afac26b4fc9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "55eaebcc-0bb8-46e5-bb2b-7afac26b4fc9", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "performer": [ - { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "683826c9-9958-41bb-8068-db0949431668" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "1972-01-21T04:21:42-08:00", - "end": "1973-01-21T04:21:42-08:00" - }, - "created": "1972-01-21T04:21:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:683826c9-9958-41bb-8068-db0949431668" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7d6bd517-ca1c-4418-8d83-db5eb534a11d" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a9cda84f-edc5-442b-abb6-45625823f948" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "446096008", - "display": "Perennial allergic rhinitis" - } - ], - "text": "Perennial allergic rhinitis" - }, - "servicedPeriod": { - "start": "1972-01-07T04:21:42-08:00", - "end": "1972-01-21T04:21:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b", - "resource": { - "resourceType": "Encounter", - "id": "4ec97cae-451d-4cd4-b818-626a34e6980b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1972-01-27T04:21:42-08:00", - "end": "1972-01-27T04:36:42-08:00" - }, - "individual": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - } - ], - "period": { - "start": "1972-01-27T04:21:42-08:00", - "end": "1972-01-27T04:36:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9b6543de-007f-4d5f-a234-48eb078df0ec", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "9b6543de-007f-4d5f-a234-48eb078df0ec", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/allergyintolerance-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/allergyintolerance-verification", - "code": "confirmed" - } - ] - }, - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "419474003", - "display": "Allergy to mould" - } - ], - "text": "Allergy to mould" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "recordedDate": "1972-01-27T04:21:42-08:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:db3bbae7-a750-465a-9b8e-ff8cdec3738c", - "resource": { - "resourceType": "CareTeam", - "id": "db3bbae7-a750-465a-9b8e-ff8cdec3738c", - "status": "active", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" - }, - "period": { - "start": "1972-01-27T04:21:42-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:61bf55ba-45b7-4a7c-a07f-285c81e20d82", - "resource": { - "resourceType": "CarePlan", - "id": "61bf55ba-45b7-4a7c-a07f-285c81e20d82", - "text": { - "status": "generated", - "div": "
    Care Plan for Self-care interventions (procedure).
    Activities:
    • Self-care interventions (procedure)
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "384758001", - "display": "Self-care interventions (procedure)" - } - ], - "text": "Self-care interventions (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" - }, - "period": { - "start": "1972-01-27T04:21:42-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:db3bbae7-a750-465a-9b8e-ff8cdec3738c" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58332002", - "display": "Allergy education" - } - ], - "text": "Allergy education" - }, - "status": "in-progress", - "location": { - "display": "LOWELL GENERAL HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:e2e13816-1dbc-4e1e-af9b-03a924a10218", - "resource": { - "resourceType": "Claim", - "id": "e2e13816-1dbc-4e1e-af9b-03a924a10218", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "1972-01-27T04:21:42-08:00", - "end": "1972-01-27T04:36:42-08:00" - }, - "created": "1972-01-27T04:36:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e511ccf6-e468-4f52-b87c-43ca688051f2", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e511ccf6-e468-4f52-b87c-43ca688051f2", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "performer": [ - { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e2e13816-1dbc-4e1e-af9b-03a924a10218" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "1972-01-27T04:36:42-08:00", - "end": "1973-01-27T04:36:42-08:00" - }, - "created": "1972-01-27T04:36:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e2e13816-1dbc-4e1e-af9b-03a924a10218" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "1972-01-27T04:21:42-08:00", - "end": "1972-01-27T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4ec97cae-451d-4cd4-b818-626a34e6980b" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "resource": { - "resourceType": "Organization", - "id": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC", - "telecom": [ - { - "system": "phone", - "value": "978-667-8600" - } - ], - "address": [ - { - "line": [ - "2 ANDOVER RD" - ], - "city": "BILLERICA", - "state": "MA", - "postalCode": "01821-1916", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "resource": { - "resourceType": "Practitioner", - "id": "0d5ab589-1d26-346a-b8db-e425d2e139e7", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999969899" - } - ], - "active": true, - "name": [ - { - "family": "Gerlach374", - "given": [ - "Lillia547" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Lillia547.Gerlach374@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "2 ANDOVER RD" - ], - "city": "BILLERICA", - "state": "MA", - "postalCode": "01821-1916", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56", - "resource": { - "resourceType": "Encounter", - "id": "404ae370-02ef-4ee0-b161-5700cfa03b56", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5", - "resource": { - "resourceType": "Condition", - "id": "ec171b19-3821-414f-ac72-c38e48aa26d5", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - }, - "onsetDateTime": "1993-03-08T04:21:42-08:00", - "recordedDate": "1993-03-08T04:21:42-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031", - "resource": { - "resourceType": "Condition", - "id": "a226d513-8226-4d13-9a1c-dc0833466031", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - }, - "onsetDateTime": "1993-03-08T04:21:42-08:00", - "recordedDate": "1993-03-08T04:21:42-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f2e9a131-fa38-46c6-b3c9-fc82ac20901b", - "resource": { - "resourceType": "CareTeam", - "id": "f2e9a131-fa38-46c6-b3c9-fc82ac20901b", - "status": "active", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - }, - "period": { - "start": "1993-03-08T04:21:42-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:061c6d94-1977-4e32-9ce6-25e0e2bbc96d", - "resource": { - "resourceType": "CarePlan", - "id": "061c6d94-1977-4e32-9ce6-25e0e2bbc96d", - "text": { - "status": "generated", - "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698360004", - "display": "Diabetes self management plan" - } - ], - "text": "Diabetes self management plan" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - }, - "period": { - "start": "1993-03-08T04:21:42-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:f2e9a131-fa38-46c6-b3c9-fc82ac20901b" - } - ], - "addresses": [ - { - "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "160670007", - "display": "Diabetic diet" - } - ], - "text": "Diabetic diet" - }, - "status": "in-progress", - "location": { - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "location": { - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:15bca152-8ff3-4eb5-9912-3b9756a18bff", - "resource": { - "resourceType": "Claim", - "id": "15bca152-8ff3-4eb5-9912-3b9756a18bff", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "created": "1993-03-08T04:51:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" - } - }, - { - "sequence": 2, - "diagnosisReference": { - "reference": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - }, - { - "sequence": 3, - "diagnosisSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:50b0c5d1-0fc1-48ca-beda-f1527e677d6b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "50b0c5d1-0fc1-48ca-beda-f1527e677d6b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "15bca152-8ff3-4eb5-9912-3b9756a18bff" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "1993-03-08T04:51:42-08:00", - "end": "1994-03-08T04:51:42-08:00" - }, - "created": "1993-03-08T04:51:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:15bca152-8ff3-4eb5-9912-3b9756a18bff" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:ec171b19-3821-414f-ac72-c38e48aa26d5" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - }, - { - "sequence": 2, - "diagnosisReference": { - "reference": "urn:uuid:a226d513-8226-4d13-9a1c-dc0833466031" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:404ae370-02ef-4ee0-b161-5700cfa03b56" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "servicedPeriod": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - }, - { - "sequence": 3, - "diagnosisSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "servicedPeriod": { - "start": "1993-03-08T04:21:42-08:00", - "end": "1993-03-08T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d", - "resource": { - "resourceType": "Encounter", - "id": "c3eba345-bfef-45ac-ae1a-2d83edf5ba6d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-01-16T04:21:42-08:00", - "end": "2012-01-16T04:36:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2012-01-16T04:21:42-08:00", - "end": "2012-01-16T04:36:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:405fba50-1d74-4d41-9708-ef2a71508c84", - "resource": { - "resourceType": "Observation", - "id": "405fba50-1d74-4d41-9708-ef2a71508c84", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f645c75e-0677-4f8b-8e21-8738ae538a50", - "resource": { - "resourceType": "Observation", - "id": "f645c75e-0677-4f8b-8e21-8738ae538a50", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fca828f7-bc07-4501-bc02-764e2777f07a", - "resource": { - "resourceType": "Observation", - "id": "fca828f7-bc07-4501-bc02-764e2777f07a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 91.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:720c5016-5da7-4c9f-837a-296672887bb3", - "resource": { - "resourceType": "Observation", - "id": "720c5016-5da7-4c9f-837a-296672887bb3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 30.04, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25277c04-7d44-4427-8eb6-4530fea701eb", - "resource": { - "resourceType": "Observation", - "id": "25277c04-7d44-4427-8eb6-4530fea701eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e99fe67-41d1-410f-bd66-bb19a0a0c641", - "resource": { - "resourceType": "Observation", - "id": "6e99fe67-41d1-410f-bd66-bb19a0a0c641", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28c85ce0-468a-4ebb-98a0-3bd9e95553e1", - "resource": { - "resourceType": "Observation", - "id": "28c85ce0-468a-4ebb-98a0-3bd9e95553e1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a151e5ae-394a-4c28-97d0-9d917d08ea02", - "resource": { - "resourceType": "Observation", - "id": "a151e5ae-394a-4c28-97d0-9d917d08ea02", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 66.93, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:37156042-e715-464a-803c-27662b1a2ba2", - "resource": { - "resourceType": "Observation", - "id": "37156042-e715-464a-803c-27662b1a2ba2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.01, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68b78785-c162-45c4-8258-349e5164235b", - "resource": { - "resourceType": "Observation", - "id": "68b78785-c162-45c4-8258-349e5164235b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.22, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", - "resource": { - "resourceType": "Observation", - "id": "f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 8.73, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", - "resource": { - "resourceType": "Observation", - "id": "dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 139.66, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7fe9e73d-f0af-4447-82a6-4f748bff33b8", - "resource": { - "resourceType": "Observation", - "id": "7fe9e73d-f0af-4447-82a6-4f748bff33b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.22, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:516f75a5-6ea6-4143-8321-0b55f2ca890b", - "resource": { - "resourceType": "Observation", - "id": "516f75a5-6ea6-4143-8321-0b55f2ca890b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 109.91, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff491da6-c0af-40ea-9240-7a63af4828e0", - "resource": { - "resourceType": "Observation", - "id": "ff491da6-c0af-40ea-9240-7a63af4828e0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 21.89, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6deedc99-ea9b-43a9-bfb4-af8fd748c079", - "resource": { - "resourceType": "Observation", - "id": "6deedc99-ea9b-43a9-bfb4-af8fd748c079", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4631f98a-74fc-4daa-b5ff-b81a76035342", - "resource": { - "resourceType": "Observation", - "id": "4631f98a-74fc-4daa-b5ff-b81a76035342", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "valueQuantity": { - "value": 5.94, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d43e29d7-147f-4e85-aceb-bef5252883aa", - "resource": { - "resourceType": "Immunization", - "id": "d43e29d7-147f-4e85-aceb-bef5252883aa", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "occurrenceDateTime": "2012-01-16T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ff140c3e-a905-47b2-af10-d6e4722c1274", - "resource": { - "resourceType": "DiagnosticReport", - "id": "ff140c3e-a905-47b2-af10-d6e4722c1274", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - }, - "effectiveDateTime": "2012-01-16T04:21:42-08:00", - "issued": "2012-01-16T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:a151e5ae-394a-4c28-97d0-9d917d08ea02", - "display": "Glucose" - }, - { - "reference": "urn:uuid:37156042-e715-464a-803c-27662b1a2ba2", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:68b78785-c162-45c4-8258-349e5164235b", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:f1345d6f-142d-4bf4-a07c-ea95fb75c3b5", - "display": "Calcium" - }, - { - "reference": "urn:uuid:dc13c21b-1cd4-4d4d-b5a3-5bae207cccdb", - "display": "Sodium" - }, - { - "reference": "urn:uuid:7fe9e73d-f0af-4447-82a6-4f748bff33b8", - "display": "Potassium" - }, - { - "reference": "urn:uuid:516f75a5-6ea6-4143-8321-0b55f2ca890b", - "display": "Chloride" - }, - { - "reference": "urn:uuid:ff491da6-c0af-40ea-9240-7a63af4828e0", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b62dbf14-d8ea-435a-9c33-9684373aa85a", - "resource": { - "resourceType": "Claim", - "id": "b62dbf14-d8ea-435a-9c33-9684373aa85a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2012-01-16T04:21:42-08:00", - "end": "2012-01-16T04:36:42-08:00" - }, - "created": "2012-01-16T04:36:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d43e29d7-147f-4e85-aceb-bef5252883aa" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:398fea0b-50cd-4d00-b362-287d02aa20db", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "398fea0b-50cd-4d00-b362-287d02aa20db", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b62dbf14-d8ea-435a-9c33-9684373aa85a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2012-01-16T04:36:42-08:00", - "end": "2013-01-16T04:36:42-08:00" - }, - "created": "2012-01-16T04:36:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b62dbf14-d8ea-435a-9c33-9684373aa85a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2012-01-16T04:21:42-08:00", - "end": "2012-01-16T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c3eba345-bfef-45ac-ae1a-2d83edf5ba6d" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2012-01-16T04:21:42-08:00", - "end": "2012-01-16T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae", - "resource": { - "resourceType": "Encounter", - "id": "29287adc-a638-4419-bb56-80b654c954ae", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-01-20T04:21:42-08:00", - "end": "2014-01-20T04:36:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2014-01-20T04:21:42-08:00", - "end": "2014-01-20T04:36:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5d32ca1a-944f-4668-920c-7d427bb582c2", - "resource": { - "resourceType": "Observation", - "id": "5d32ca1a-944f-4668-920c-7d427bb582c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40b4a2a6-0371-4ac8-a2aa-98a0736a22b7", - "resource": { - "resourceType": "Observation", - "id": "40b4a2a6-0371-4ac8-a2aa-98a0736a22b7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:297b8a08-5868-4c52-9104-c08d3e3661dd", - "resource": { - "resourceType": "Observation", - "id": "297b8a08-5868-4c52-9104-c08d3e3661dd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 91.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2fffaa51-499a-46c0-9d76-68620f71c829", - "resource": { - "resourceType": "Observation", - "id": "2fffaa51-499a-46c0-9d76-68620f71c829", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 30.04, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e890450b-52b2-4f0c-9ef9-b3e344476770", - "resource": { - "resourceType": "Observation", - "id": "e890450b-52b2-4f0c-9ef9-b3e344476770", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34bf394d-0d3d-480f-9ef4-31e04cc3ea93", - "resource": { - "resourceType": "Observation", - "id": "34bf394d-0d3d-480f-9ef4-31e04cc3ea93", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e1695f4-744c-4c4b-9a9c-04b9fab1625c", - "resource": { - "resourceType": "Observation", - "id": "8e1695f4-744c-4c4b-9a9c-04b9fab1625c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", - "resource": { - "resourceType": "Observation", - "id": "b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 76.05, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0b357ed-4ce0-4ae4-97fb-61c179d777f6", - "resource": { - "resourceType": "Observation", - "id": "b0b357ed-4ce0-4ae4-97fb-61c179d777f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 13.88, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69fcb49e-b46c-45b1-bd17-aa7c5792d929", - "resource": { - "resourceType": "Observation", - "id": "69fcb49e-b46c-45b1-bd17-aa7c5792d929", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.3, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:190af0b9-71e7-4668-b685-772ebfca658f", - "resource": { - "resourceType": "Observation", - "id": "190af0b9-71e7-4668-b685-772ebfca658f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 8.62, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6f8610c-c722-45b8-a15d-5c11d8349fa7", - "resource": { - "resourceType": "Observation", - "id": "c6f8610c-c722-45b8-a15d-5c11d8349fa7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 137.49, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", - "resource": { - "resourceType": "Observation", - "id": "8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.49, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c7fe2b8-5a5d-4545-a76a-44791bb28871", - "resource": { - "resourceType": "Observation", - "id": "3c7fe2b8-5a5d-4545-a76a-44791bb28871", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 107.65, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d86173e0-44ba-4d36-bde6-02329d4d46c4", - "resource": { - "resourceType": "Observation", - "id": "d86173e0-44ba-4d36-bde6-02329d4d46c4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 22.57, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67dfe155-480d-4360-8f8e-46a2524e978b", - "resource": { - "resourceType": "Observation", - "id": "67dfe155-480d-4360-8f8e-46a2524e978b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 166, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:94927635-d2ed-4e7a-82bb-baf45c33bf32", - "resource": { - "resourceType": "Observation", - "id": "94927635-d2ed-4e7a-82bb-baf45c33bf32", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 136.52, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f4068fbe-56be-480f-b319-d953ca7f0204", - "resource": { - "resourceType": "Observation", - "id": "f4068fbe-56be-480f-b319-d953ca7f0204", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 59.83, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd747217-d486-408d-94a1-400e8409eec4", - "resource": { - "resourceType": "Observation", - "id": "cd747217-d486-408d-94a1-400e8409eec4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 78.87, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ab25180-451c-4a04-921f-5f53ab235cbc", - "resource": { - "resourceType": "Observation", - "id": "6ab25180-451c-4a04-921f-5f53ab235cbc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 6.6162, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:57bd55ad-75f5-4b90-84e8-3971e44f18af", - "resource": { - "resourceType": "Observation", - "id": "57bd55ad-75f5-4b90-84e8-3971e44f18af", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.0422, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:768eddaa-7c37-494f-861d-ede7b432cd22", - "resource": { - "resourceType": "Observation", - "id": "768eddaa-7c37-494f-861d-ede7b432cd22", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 12.795, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", - "resource": { - "resourceType": "Observation", - "id": "6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 40.741, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", - "resource": { - "resourceType": "Observation", - "id": "a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 89.664, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9b3be03f-2810-4122-8487-1a5d01db4870", - "resource": { - "resourceType": "Observation", - "id": "9b3be03f-2810-4122-8487-1a5d01db4870", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 30.462, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:373801c4-9966-44ad-8e9a-0c99e9cad240", - "resource": { - "resourceType": "Observation", - "id": "373801c4-9966-44ad-8e9a-0c99e9cad240", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 33.524, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9db75a17-a326-4201-a8fa-e2d4119fe5cf", - "resource": { - "resourceType": "Observation", - "id": "9db75a17-a326-4201-a8fa-e2d4119fe5cf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 45.737, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b47d7b7b-7d7e-4051-b511-34fbf912c8c1", - "resource": { - "resourceType": "Observation", - "id": "b47d7b7b-7d7e-4051-b511-34fbf912c8c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 357.81, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:19f52b10-41c6-4d5d-ad2a-7cad3017b343", - "resource": { - "resourceType": "Observation", - "id": "19f52b10-41c6-4d5d-ad2a-7cad3017b343", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 225.32, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0307fe87-f23c-4ce5-aff4-8c367e309cc5", - "resource": { - "resourceType": "Observation", - "id": "0307fe87-f23c-4ce5-aff4-8c367e309cc5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 12.091, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f4f7c83-d77b-4db9-a7ea-2317fed847ca", - "resource": { - "resourceType": "Observation", - "id": "9f4f7c83-d77b-4db9-a7ea-2317fed847ca", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1413d06e-1574-46fe-a7bb-8f266284493c", - "resource": { - "resourceType": "Observation", - "id": "1413d06e-1574-46fe-a7bb-8f266284493c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "valueQuantity": { - "value": 5.98, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d127a6e1-4568-4cd5-8b1d-0ed1c067173a", - "resource": { - "resourceType": "Immunization", - "id": "d127a6e1-4568-4cd5-8b1d-0ed1c067173a", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "occurrenceDateTime": "2014-01-20T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f66881d6-2adc-4d7c-8307-43a2a606b344", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f66881d6-2adc-4d7c-8307-43a2a606b344", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:b4ef81d5-e448-4ff2-b777-a8ce47d3b06a", - "display": "Glucose" - }, - { - "reference": "urn:uuid:b0b357ed-4ce0-4ae4-97fb-61c179d777f6", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:69fcb49e-b46c-45b1-bd17-aa7c5792d929", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:190af0b9-71e7-4668-b685-772ebfca658f", - "display": "Calcium" - }, - { - "reference": "urn:uuid:c6f8610c-c722-45b8-a15d-5c11d8349fa7", - "display": "Sodium" - }, - { - "reference": "urn:uuid:8a29c0f6-a74a-41f5-bf64-9d8d332f5e48", - "display": "Potassium" - }, - { - "reference": "urn:uuid:3c7fe2b8-5a5d-4545-a76a-44791bb28871", - "display": "Chloride" - }, - { - "reference": "urn:uuid:d86173e0-44ba-4d36-bde6-02329d4d46c4", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:a9e8955a-c4b5-4c24-9f3f-7bcd3c6f1351", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a9e8955a-c4b5-4c24-9f3f-7bcd3c6f1351", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:67dfe155-480d-4360-8f8e-46a2524e978b", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:94927635-d2ed-4e7a-82bb-baf45c33bf32", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:f4068fbe-56be-480f-b319-d953ca7f0204", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:cd747217-d486-408d-94a1-400e8409eec4", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7caa0371-603c-4302-8536-2664d2ee63cf", - "resource": { - "resourceType": "DiagnosticReport", - "id": "7caa0371-603c-4302-8536-2664d2ee63cf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - }, - "effectiveDateTime": "2014-01-20T04:21:42-08:00", - "issued": "2014-01-20T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:6ab25180-451c-4a04-921f-5f53ab235cbc", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:57bd55ad-75f5-4b90-84e8-3971e44f18af", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:768eddaa-7c37-494f-861d-ede7b432cd22", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:6e50b7c3-cc3b-41b3-8d45-a90ac3f5d598", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:a3e9b7d2-fa0c-41a7-b513-82cc67f5845e", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:9b3be03f-2810-4122-8487-1a5d01db4870", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:373801c4-9966-44ad-8e9a-0c99e9cad240", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:9db75a17-a326-4201-a8fa-e2d4119fe5cf", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:b47d7b7b-7d7e-4051-b511-34fbf912c8c1", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:19f52b10-41c6-4d5d-ad2a-7cad3017b343", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:0307fe87-f23c-4ce5-aff4-8c367e309cc5", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:3600f5ed-8ee3-4e14-9063-2a59f891eaec", - "resource": { - "resourceType": "Claim", - "id": "3600f5ed-8ee3-4e14-9063-2a59f891eaec", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2014-01-20T04:21:42-08:00", - "end": "2014-01-20T04:36:42-08:00" - }, - "created": "2014-01-20T04:36:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d127a6e1-4568-4cd5-8b1d-0ed1c067173a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:36be44ca-d517-4e01-b897-0572acff676d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "36be44ca-d517-4e01-b897-0572acff676d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3600f5ed-8ee3-4e14-9063-2a59f891eaec" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2014-01-20T04:36:42-08:00", - "end": "2015-01-20T04:36:42-08:00" - }, - "created": "2014-01-20T04:36:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3600f5ed-8ee3-4e14-9063-2a59f891eaec" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2014-01-20T04:21:42-08:00", - "end": "2014-01-20T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:29287adc-a638-4419-bb56-80b654c954ae" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2014-01-20T04:21:42-08:00", - "end": "2014-01-20T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87", - "resource": { - "resourceType": "Encounter", - "id": "314b78fa-de5b-4908-94b4-0846279d1c87", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6999b531-4296-4c14-ac10-bb8169b1f27c", - "resource": { - "resourceType": "Observation", - "id": "6999b531-4296-4c14-ac10-bb8169b1f27c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1fa124b2-1ede-426c-bb65-2c38576c10f8", - "resource": { - "resourceType": "Observation", - "id": "1fa124b2-1ede-426c-bb65-2c38576c10f8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b59c7ed7-84a4-4876-86cf-b6f0392e892e", - "resource": { - "resourceType": "Observation", - "id": "b59c7ed7-84a4-4876-86cf-b6f0392e892e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 83.6, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f995ff32-dbd4-4974-9ca0-69f073e934e5", - "resource": { - "resourceType": "Observation", - "id": "f995ff32-dbd4-4974-9ca0-69f073e934e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 27.34, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:de37dae5-7754-4fc5-81af-98c5a22d19e3", - "resource": { - "resourceType": "Observation", - "id": "de37dae5-7754-4fc5-81af-98c5a22d19e3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c25cd0da-91e0-4726-8948-80cbdabef496", - "resource": { - "resourceType": "Observation", - "id": "c25cd0da-91e0-4726-8948-80cbdabef496", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 81, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f502042b-2438-40bf-92a8-d14b37370247", - "resource": { - "resourceType": "Observation", - "id": "f502042b-2438-40bf-92a8-d14b37370247", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9572bdc-8661-449c-b863-9ab3c4276c66", - "resource": { - "resourceType": "Observation", - "id": "c9572bdc-8661-449c-b863-9ab3c4276c66", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 66.64, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", - "resource": { - "resourceType": "Observation", - "id": "d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 11.82, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aeaa13aa-b5be-49d8-82cd-61814e0f6760", - "resource": { - "resourceType": "Observation", - "id": "aeaa13aa-b5be-49d8-82cd-61814e0f6760", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.07, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", - "resource": { - "resourceType": "Observation", - "id": "f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ff4331b-a76b-4862-9491-7200d6fb17eb", - "resource": { - "resourceType": "Observation", - "id": "0ff4331b-a76b-4862-9491-7200d6fb17eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 140.55, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5fada0a8-adc2-4abd-9f87-fa575582bd43", - "resource": { - "resourceType": "Observation", - "id": "5fada0a8-adc2-4abd-9f87-fa575582bd43", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.25, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:144ac2fd-269c-4b47-bddb-d0605f0dfcef", - "resource": { - "resourceType": "Observation", - "id": "144ac2fd-269c-4b47-bddb-d0605f0dfcef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 109.69, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:735a868b-f60f-4522-b417-5fe4e12785b3", - "resource": { - "resourceType": "Observation", - "id": "735a868b-f60f-4522-b417-5fe4e12785b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 21.45, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33fb567d-a571-4257-973a-52d756b4c36e", - "resource": { - "resourceType": "Observation", - "id": "33fb567d-a571-4257-973a-52d756b4c36e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4bb9225-ac6c-4df9-b7a0-4d31027e72f3", - "resource": { - "resourceType": "Observation", - "id": "d4bb9225-ac6c-4df9-b7a0-4d31027e72f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "valueQuantity": { - "value": 6.24, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e86956a-2972-4fd4-b3ea-f54290189e67", - "resource": { - "resourceType": "Procedure", - "id": "7e86956a-2972-4fd4-b3ea-f54290189e67", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "performedPeriod": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:36:42-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7d52685e-a9a5-4108-b79b-2195600e79d7", - "resource": { - "resourceType": "Immunization", - "id": "7d52685e-a9a5-4108-b79b-2195600e79d7", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "occurrenceDateTime": "2016-01-25T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1126c7df-b0c2-4fad-94d4-69f24207539e", - "resource": { - "resourceType": "DiagnosticReport", - "id": "1126c7df-b0c2-4fad-94d4-69f24207539e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - }, - "effectiveDateTime": "2016-01-25T04:21:42-08:00", - "issued": "2016-01-25T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:c9572bdc-8661-449c-b863-9ab3c4276c66", - "display": "Glucose" - }, - { - "reference": "urn:uuid:d10fc0ed-ef82-42c5-9b1b-437fe2d8db58", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:aeaa13aa-b5be-49d8-82cd-61814e0f6760", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:f84378d3-2aa6-4b91-89fb-0ec90eddc6fe", - "display": "Calcium" - }, - { - "reference": "urn:uuid:0ff4331b-a76b-4862-9491-7200d6fb17eb", - "display": "Sodium" - }, - { - "reference": "urn:uuid:5fada0a8-adc2-4abd-9f87-fa575582bd43", - "display": "Potassium" - }, - { - "reference": "urn:uuid:144ac2fd-269c-4b47-bddb-d0605f0dfcef", - "display": "Chloride" - }, - { - "reference": "urn:uuid:735a868b-f60f-4522-b417-5fe4e12785b3", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40", - "resource": { - "resourceType": "Claim", - "id": "c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "created": "2016-01-25T04:51:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7d52685e-a9a5-4108-b79b-2195600e79d7" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7e86956a-2972-4fd4-b3ea-f54290189e67" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 456.86, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3cd92632-7d1b-4f78-9cb2-f85ed0b81966", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3cd92632-7d1b-4f78-9cb2-f85ed0b81966", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2016-01-25T04:51:42-08:00", - "end": "2017-01-25T04:51:42-08:00" - }, - "created": "2016-01-25T04:51:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:c9cbfb66-3e18-4de2-bcbb-d6b13e95fc40" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:314b78fa-de5b-4908-94b4-0846279d1c87" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-25T04:21:42-08:00", - "end": "2016-01-25T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 456.86, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 91.37200000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 365.48800000000006, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 456.86, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 456.86, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 477.90400000000005, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3", - "resource": { - "resourceType": "Encounter", - "id": "a8fd8f65-82a3-4b48-890e-e6a1a69489e3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-11-27T04:21:42-08:00", - "end": "2017-11-27T04:36:42-08:00" - }, - "individual": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - } - ], - "period": { - "start": "2017-11-27T04:21:42-08:00", - "end": "2017-11-27T04:36:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6b537404-85fa-4571-94c7-b2aac60aa56d", - "resource": { - "resourceType": "Immunization", - "id": "6b537404-85fa-4571-94c7-b2aac60aa56d", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" - }, - "occurrenceDateTime": "2017-11-27T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:baeb3485-524a-445e-9507-cb8d7faed25c", - "resource": { - "resourceType": "Claim", - "id": "baeb3485-524a-445e-9507-cb8d7faed25c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2017-11-27T04:21:42-08:00", - "end": "2017-11-27T04:36:42-08:00" - }, - "created": "2017-11-27T04:36:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:6b537404-85fa-4571-94c7-b2aac60aa56d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1f8bbb1d-49e9-4fe8-8340-3dfbdcbbc868", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1f8bbb1d-49e9-4fe8-8340-3dfbdcbbc868", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "performer": [ - { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "baeb3485-524a-445e-9507-cb8d7faed25c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2017-11-27T04:36:42-08:00", - "end": "2018-11-27T04:36:42-08:00" - }, - "created": "2017-11-27T04:36:42-08:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:baeb3485-524a-445e-9507-cb8d7faed25c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "2017-11-27T04:21:42-08:00", - "end": "2017-11-27T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a8fd8f65-82a3-4b48-890e-e6a1a69489e3" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2017-11-27T04:21:42-08:00", - "end": "2017-11-27T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317", - "resource": { - "resourceType": "Encounter", - "id": "1a87fe76-cbaa-49a3-9256-db5e5962d317", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "individual": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - } - ], - "period": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff", - "resource": { - "resourceType": "Condition", - "id": "b19cc824-fc9f-4034-9dbe-3232028985ff", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "onsetDateTime": "2017-11-21T04:21:42-08:00", - "abatementDateTime": "2017-12-01T04:21:42-08:00", - "recordedDate": "2017-11-21T04:21:42-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:1c4ee58b-d323-470a-ae25-739d67dc4b5d", - "resource": { - "resourceType": "Observation", - "id": "1c4ee58b-d323-470a-ae25-739d67dc4b5d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - }, - { - "system": "http://loinc.org", - "code": "8331-1", - "display": "Oral temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-21T04:21:42-08:00", - "issued": "2017-11-21T04:21:42.840-08:00", - "valueQuantity": { - "value": 37.019, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:650b0fe7-f02b-44f0-b504-1043b3e937c4", - "resource": { - "resourceType": "Observation", - "id": "650b0fe7-f02b-44f0-b504-1043b3e937c4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b4396c34-bda6-43f6-8b57-c1ae9d155ea8", - "resource": { - "resourceType": "Observation", - "id": "b4396c34-bda6-43f6-8b57-c1ae9d155ea8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:45af5249-feaf-435e-862d-228ae3377222", - "resource": { - "resourceType": "Observation", - "id": "45af5249-feaf-435e-862d-228ae3377222", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 86.6, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8032b18d-63e8-4694-8b16-e5f5ac8e87ce", - "resource": { - "resourceType": "Observation", - "id": "8032b18d-63e8-4694-8b16-e5f5ac8e87ce", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 28.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4ea6b234-08d0-4125-9816-691bc3a44566", - "resource": { - "resourceType": "Observation", - "id": "4ea6b234-08d0-4125-9816-691bc3a44566", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 128, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49d20692-95c1-4a62-86d7-6f15be0a2472", - "resource": { - "resourceType": "Observation", - "id": "49d20692-95c1-4a62-86d7-6f15be0a2472", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2cecf73f-273c-474e-ab83-48d29e79645c", - "resource": { - "resourceType": "Observation", - "id": "2cecf73f-273c-474e-ab83-48d29e79645c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d996a3cd-7ca4-4667-be64-5cd99dc952b0", - "resource": { - "resourceType": "Observation", - "id": "d996a3cd-7ca4-4667-be64-5cd99dc952b0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 96.52, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:80f09d42-3a48-43dd-bc4f-b5544c8b4df0", - "resource": { - "resourceType": "Observation", - "id": "80f09d42-3a48-43dd-bc4f-b5544c8b4df0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 11.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36854602-1113-4729-83d0-9d32df3c8654", - "resource": { - "resourceType": "Observation", - "id": "36854602-1113-4729-83d0-9d32df3c8654", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", - "resource": { - "resourceType": "Observation", - "id": "9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.1, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4f1e36e-2aef-4388-b243-4a54558d0ff8", - "resource": { - "resourceType": "Observation", - "id": "c4f1e36e-2aef-4388-b243-4a54558d0ff8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 140.35, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", - "resource": { - "resourceType": "Observation", - "id": "10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.77, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dbc9df23-758e-4099-b808-a610666fd6e8", - "resource": { - "resourceType": "Observation", - "id": "dbc9df23-758e-4099-b808-a610666fd6e8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 105.2, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70c0597b-1b9c-4322-9d11-111de2d4f8df", - "resource": { - "resourceType": "Observation", - "id": "70c0597b-1b9c-4322-9d11-111de2d4f8df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 26.82, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cbcfa27d-84c6-479a-8607-7ce01a2174ee", - "resource": { - "resourceType": "Observation", - "id": "cbcfa27d-84c6-479a-8607-7ce01a2174ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 198.15, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", - "resource": { - "resourceType": "Observation", - "id": "ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 131.29, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f6cd5cc8-c32b-45e3-87c9-96a19087809a", - "resource": { - "resourceType": "Observation", - "id": "f6cd5cc8-c32b-45e3-87c9-96a19087809a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 94.57, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83a8c0ee-d80a-4162-86b6-3c371c91e79f", - "resource": { - "resourceType": "Observation", - "id": "83a8c0ee-d80a-4162-86b6-3c371c91e79f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 77.32, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac157b02-6bf9-4240-8250-72108a0f907f", - "resource": { - "resourceType": "Observation", - "id": "ac157b02-6bf9-4240-8250-72108a0f907f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d902f66f-14e3-4f38-bfcb-c777e8934668", - "resource": { - "resourceType": "Observation", - "id": "d902f66f-14e3-4f38-bfcb-c777e8934668", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "valueQuantity": { - "value": 6.08, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7b24c86c-b99f-4227-8555-6a87b3285f87", - "resource": { - "resourceType": "Procedure", - "id": "7b24c86c-b99f-4227-8555-6a87b3285f87", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "performedPeriod": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:36:42-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff", - "display": "Acute viral pharyngitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:82fa7941-2a98-4912-9b68-7107a1612d23", - "resource": { - "resourceType": "DiagnosticReport", - "id": "82fa7941-2a98-4912-9b68-7107a1612d23", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:d996a3cd-7ca4-4667-be64-5cd99dc952b0", - "display": "Glucose" - }, - { - "reference": "urn:uuid:80f09d42-3a48-43dd-bc4f-b5544c8b4df0", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:36854602-1113-4729-83d0-9d32df3c8654", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:9aa5b3e2-29da-42dd-9d72-6ceb3abf8f3d", - "display": "Calcium" - }, - { - "reference": "urn:uuid:c4f1e36e-2aef-4388-b243-4a54558d0ff8", - "display": "Sodium" - }, - { - "reference": "urn:uuid:10b046e6-6f8f-4bc7-a599-20ce3e1a4c8b", - "display": "Potassium" - }, - { - "reference": "urn:uuid:dbc9df23-758e-4099-b808-a610666fd6e8", - "display": "Chloride" - }, - { - "reference": "urn:uuid:70c0597b-1b9c-4322-9d11-111de2d4f8df", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:2fff41a0-1ed4-4a68-bccd-7837648fb420", - "resource": { - "resourceType": "DiagnosticReport", - "id": "2fff41a0-1ed4-4a68-bccd-7837648fb420", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - }, - "effectiveDateTime": "2017-11-27T04:21:42-08:00", - "issued": "2017-11-27T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:cbcfa27d-84c6-479a-8607-7ce01a2174ee", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:ab80fd54-fbcc-484a-b1d1-7c8d0bb1c14b", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:f6cd5cc8-c32b-45e3-87c9-96a19087809a", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:83a8c0ee-d80a-4162-86b6-3c371c91e79f", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:3632e83a-3b9b-409c-bf8f-213910fc1856", - "resource": { - "resourceType": "Claim", - "id": "3632e83a-3b9b-409c-bf8f-213910fc1856", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "created": "2017-11-21T04:51:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7b24c86c-b99f-4227-8555-6a87b3285f87" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "net": { - "value": 2515.75, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9e902b1d-5977-4010-8540-f9cddbbbb27a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9e902b1d-5977-4010-8540-f9cddbbbb27a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "performer": [ - { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3632e83a-3b9b-409c-bf8f-213910fc1856" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2017-11-21T04:51:42-08:00", - "end": "2018-11-21T04:51:42-08:00" - }, - "created": "2017-11-21T04:51:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3632e83a-3b9b-409c-bf8f-213910fc1856" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b19cc824-fc9f-4034-9dbe-3232028985ff" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1a87fe76-cbaa-49a3-9256-db5e5962d317" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "servicedPeriod": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "servicedPeriod": { - "start": "2017-11-21T04:21:42-08:00", - "end": "2017-11-21T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2515.75, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 503.15000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2012.6000000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2515.75, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2515.75, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 2012.6000000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294", - "resource": { - "resourceType": "Encounter", - "id": "e02df952-5c01-4e8b-8cd1-e5f3e7ae8294", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:737add4e-d92a-4bfc-8282-6689449451d7", - "resource": { - "resourceType": "Observation", - "id": "737add4e-d92a-4bfc-8282-6689449451d7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6659c095-2852-4c10-802a-7c8feac75ec3", - "resource": { - "resourceType": "Observation", - "id": "6659c095-2852-4c10-802a-7c8feac75ec3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f74337f3-7d3b-497d-8020-5dd8ef4fc5a0", - "resource": { - "resourceType": "Observation", - "id": "f74337f3-7d3b-497d-8020-5dd8ef4fc5a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 86.9, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7670f173-806f-40c6-8f27-4e7a35447de8", - "resource": { - "resourceType": "Observation", - "id": "7670f173-806f-40c6-8f27-4e7a35447de8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 28.43, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77f0c372-680e-4dff-8b4f-aef5fa2f528b", - "resource": { - "resourceType": "Observation", - "id": "77f0c372-680e-4dff-8b4f-aef5fa2f528b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 135, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d52b4ba4-0818-4a1a-92c7-0a65190125df", - "resource": { - "resourceType": "Observation", - "id": "d52b4ba4-0818-4a1a-92c7-0a65190125df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 66, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:64c8ea7d-4177-4a78-8066-a5ad2fb01aea", - "resource": { - "resourceType": "Observation", - "id": "64c8ea7d-4177-4a78-8066-a5ad2fb01aea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8572f071-3410-4162-b2d4-4e549320795e", - "resource": { - "resourceType": "Observation", - "id": "8572f071-3410-4162-b2d4-4e549320795e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 67.19, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d8044db-9081-4bc2-988e-9f5c54fbdc97", - "resource": { - "resourceType": "Observation", - "id": "5d8044db-9081-4bc2-988e-9f5c54fbdc97", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 14.93, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8bc31c89-5920-4d8f-8bb2-b9a23400b192", - "resource": { - "resourceType": "Observation", - "id": "8bc31c89-5920-4d8f-8bb2-b9a23400b192", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:abda425b-5dcf-4e5b-ab29-671cc5eb4a76", - "resource": { - "resourceType": "Observation", - "id": "abda425b-5dcf-4e5b-ab29-671cc5eb4a76", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.8, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4d938e2f-eafd-452d-8683-2b657dc7c8ac", - "resource": { - "resourceType": "Observation", - "id": "4d938e2f-eafd-452d-8683-2b657dc7c8ac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 138.61, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:04d489bd-f171-4c6b-9f37-141934d09762", - "resource": { - "resourceType": "Observation", - "id": "04d489bd-f171-4c6b-9f37-141934d09762", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.08, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4cce0680-8bac-4b52-8b85-9ada6816eda4", - "resource": { - "resourceType": "Observation", - "id": "4cce0680-8bac-4b52-8b85-9ada6816eda4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 109.63, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:20ea4432-3403-4331-b6e2-d8a579c43240", - "resource": { - "resourceType": "Observation", - "id": "20ea4432-3403-4331-b6e2-d8a579c43240", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 28.45, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ba5431d-2b30-4cba-930c-0d36e31b3cb7", - "resource": { - "resourceType": "Observation", - "id": "6ba5431d-2b30-4cba-930c-0d36e31b3cb7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:85ebea82-7fc5-4775-afd1-ef0d2b801ae1", - "resource": { - "resourceType": "Immunization", - "id": "85ebea82-7fc5-4775-afd1-ef0d2b801ae1", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "occurrenceDateTime": "2018-01-01T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7d95bd33-c7e3-479e-949d-323110979ba7", - "resource": { - "resourceType": "Immunization", - "id": "7d95bd33-c7e3-479e-949d-323110979ba7", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "occurrenceDateTime": "2018-01-01T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:114dfa31-82b4-4564-a64e-0c94af7c88c0", - "resource": { - "resourceType": "DiagnosticReport", - "id": "114dfa31-82b4-4564-a64e-0c94af7c88c0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:8572f071-3410-4162-b2d4-4e549320795e", - "display": "Glucose" - }, - { - "reference": "urn:uuid:5d8044db-9081-4bc2-988e-9f5c54fbdc97", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:8bc31c89-5920-4d8f-8bb2-b9a23400b192", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:abda425b-5dcf-4e5b-ab29-671cc5eb4a76", - "display": "Calcium" - }, - { - "reference": "urn:uuid:4d938e2f-eafd-452d-8683-2b657dc7c8ac", - "display": "Sodium" - }, - { - "reference": "urn:uuid:04d489bd-f171-4c6b-9f37-141934d09762", - "display": "Potassium" - }, - { - "reference": "urn:uuid:4cce0680-8bac-4b52-8b85-9ada6816eda4", - "display": "Chloride" - }, - { - "reference": "urn:uuid:20ea4432-3403-4331-b6e2-d8a579c43240", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:bb5824dd-797a-4f24-946d-a7e710e3a391", - "resource": { - "resourceType": "Claim", - "id": "bb5824dd-797a-4f24-946d-a7e710e3a391", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "created": "2018-01-01T04:36:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:85ebea82-7fc5-4775-afd1-ef0d2b801ae1" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7d95bd33-c7e3-479e-949d-323110979ba7" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:96cc857a-9127-43f9-829f-1ba3ad1af8ec", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "96cc857a-9127-43f9-829f-1ba3ad1af8ec", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bb5824dd-797a-4f24-946d-a7e710e3a391" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2018-01-01T04:36:42-08:00", - "end": "2019-01-01T04:36:42-08:00" - }, - "created": "2018-01-01T04:36:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:bb5824dd-797a-4f24-946d-a7e710e3a391" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e02df952-5c01-4e8b-8cd1-e5f3e7ae8294" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 224.83200000000002, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e", - "resource": { - "resourceType": "Encounter", - "id": "576347df-eecc-4bef-b430-8117181dcd6e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "individual": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "display": "Dr. Franklin857 Tillman293" - } - } - ], - "period": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:33355ff7-c208-4cd5-a114-da4d67285f20", - "resource": { - "resourceType": "Observation", - "id": "33355ff7-c208-4cd5-a114-da4d67285f20", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" - }, - "effectiveDateTime": "2018-01-01T04:21:42-08:00", - "issued": "2018-01-01T04:21:42.840-08:00", - "valueQuantity": { - "value": 5.97, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:310eae17-0e5b-421e-bf42-53765954b278", - "resource": { - "resourceType": "Procedure", - "id": "310eae17-0e5b-421e-bf42-53765954b278", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" - }, - "performedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:04:42-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:79bc6b37-fd91-4870-995c-d931221e0652", - "resource": { - "resourceType": "Procedure", - "id": "79bc6b37-fd91-4870-995c-d931221e0652", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" - }, - "performedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T04:36:42-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d370258b-0a72-47dd-b762-dd61b29dcd63", - "resource": { - "resourceType": "Claim", - "id": "d370258b-0a72-47dd-b762-dd61b29dcd63", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "created": "2018-01-01T05:19:42-08:00", - "provider": { - "reference": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "display": "LOWELL GENERAL HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:310eae17-0e5b-421e-bf42-53765954b278" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:79bc6b37-fd91-4870-995c-d931221e0652" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 9754.14, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 529.07, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1bca55b3-5f6b-4168-aa47-cdf51a576d98", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1bca55b3-5f6b-4168-aa47-cdf51a576d98", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "performer": [ - { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d370258b-0a72-47dd-b762-dd61b29dcd63" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2018-01-01T05:19:42-08:00", - "end": "2019-01-01T05:19:42-08:00" - }, - "created": "2018-01-01T05:19:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:d370258b-0a72-47dd-b762-dd61b29dcd63" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:576347df-eecc-4bef-b430-8117181dcd6e" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 9754.14, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1950.828, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7803.312, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 9754.14, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 9754.14, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2018-01-01T04:21:42-08:00", - "end": "2018-01-01T05:19:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 529.07, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 105.81400000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 423.2560000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 529.07, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 529.07, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 8226.568, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1", - "resource": { - "resourceType": "Encounter", - "id": "a721f251-d45e-40c1-8da1-1edf0b8e92f1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:07719b28-0951-48e4-a494-f720fc508653", - "resource": { - "resourceType": "Observation", - "id": "07719b28-0951-48e4-a494-f720fc508653", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfe107b7-8e45-47e3-a30c-aab5562c2915", - "resource": { - "resourceType": "Observation", - "id": "cfe107b7-8e45-47e3-a30c-aab5562c2915", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28ffe479-e81b-4e3f-9349-9d911afdd5ba", - "resource": { - "resourceType": "Observation", - "id": "28ffe479-e81b-4e3f-9349-9d911afdd5ba", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 89.2, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fe5259ca-c339-4876-bfdf-42c80a79eaf7", - "resource": { - "resourceType": "Observation", - "id": "fe5259ca-c339-4876-bfdf-42c80a79eaf7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 29.19, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2741101d-771b-45cc-a700-6c0246e183f5", - "resource": { - "resourceType": "Observation", - "id": "2741101d-771b-45cc-a700-6c0246e183f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:156e6b08-063e-43bf-ac47-06df7bb1027b", - "resource": { - "resourceType": "Observation", - "id": "156e6b08-063e-43bf-ac47-06df7bb1027b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 67, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c93dde1e-33f8-421b-8b95-85f266916b27", - "resource": { - "resourceType": "Observation", - "id": "c93dde1e-33f8-421b-8b95-85f266916b27", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2f00950-67ef-43e2-899d-790510611905", - "resource": { - "resourceType": "Observation", - "id": "e2f00950-67ef-43e2-899d-790510611905", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 82.75, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:63fe2f46-9f03-42a9-9eff-29721a754a1d", - "resource": { - "resourceType": "Observation", - "id": "63fe2f46-9f03-42a9-9eff-29721a754a1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 7.22, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c1d13db-9b7f-4552-b39a-16d45483dc75", - "resource": { - "resourceType": "Observation", - "id": "0c1d13db-9b7f-4552-b39a-16d45483dc75", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.15, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3dad7698-39da-4ce9-af7a-3da4687945b9", - "resource": { - "resourceType": "Observation", - "id": "3dad7698-39da-4ce9-af7a-3da4687945b9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.84, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fedd1f46-d752-4ac8-8d7f-fe5059d941ef", - "resource": { - "resourceType": "Observation", - "id": "fedd1f46-d752-4ac8-8d7f-fe5059d941ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 140.73, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:448f2d10-75e9-4431-b73b-6237dead796b", - "resource": { - "resourceType": "Observation", - "id": "448f2d10-75e9-4431-b73b-6237dead796b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.93, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", - "resource": { - "resourceType": "Observation", - "id": "6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 106.98, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab276e3d-4f4e-495b-be7f-b3e9287a752b", - "resource": { - "resourceType": "Observation", - "id": "ab276e3d-4f4e-495b-be7f-b3e9287a752b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 27.25, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bc99199a-eee1-499e-8bf0-76ab223ae204", - "resource": { - "resourceType": "Observation", - "id": "bc99199a-eee1-499e-8bf0-76ab223ae204", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:195fbf1b-90f3-4c1d-9785-139d3b4524ff", - "resource": { - "resourceType": "Observation", - "id": "195fbf1b-90f3-4c1d-9785-139d3b4524ff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "valueQuantity": { - "value": 5.83, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d0668a8-2b39-4a87-8c13-0bbf20a09d04", - "resource": { - "resourceType": "Procedure", - "id": "9d0668a8-2b39-4a87-8c13-0bbf20a09d04", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "performedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:36:42-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:930c7097-a336-4d01-9c6b-ff038011ffe1", - "resource": { - "resourceType": "Immunization", - "id": "930c7097-a336-4d01-9c6b-ff038011ffe1", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "occurrenceDateTime": "2019-01-07T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:22e25df5-de3c-45da-959d-24a25a8553d1", - "resource": { - "resourceType": "Immunization", - "id": "22e25df5-de3c-45da-959d-24a25a8553d1", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "occurrenceDateTime": "2019-01-07T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1959d580-9164-4a51-84cf-a374c3b97d22", - "resource": { - "resourceType": "Immunization", - "id": "1959d580-9164-4a51-84cf-a374c3b97d22", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "occurrenceDateTime": "2019-01-07T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c8799c64-5ebd-4133-8cfa-b521abaa6387", - "resource": { - "resourceType": "DiagnosticReport", - "id": "c8799c64-5ebd-4133-8cfa-b521abaa6387", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - }, - "effectiveDateTime": "2019-01-07T04:21:42-08:00", - "issued": "2019-01-07T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:e2f00950-67ef-43e2-899d-790510611905", - "display": "Glucose" - }, - { - "reference": "urn:uuid:63fe2f46-9f03-42a9-9eff-29721a754a1d", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:0c1d13db-9b7f-4552-b39a-16d45483dc75", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:3dad7698-39da-4ce9-af7a-3da4687945b9", - "display": "Calcium" - }, - { - "reference": "urn:uuid:fedd1f46-d752-4ac8-8d7f-fe5059d941ef", - "display": "Sodium" - }, - { - "reference": "urn:uuid:448f2d10-75e9-4431-b73b-6237dead796b", - "display": "Potassium" - }, - { - "reference": "urn:uuid:6dd8d9b8-2923-45ef-bb78-873cf7e6c3f6", - "display": "Chloride" - }, - { - "reference": "urn:uuid:ab276e3d-4f4e-495b-be7f-b3e9287a752b", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0f9bc1f1-05bc-4172-8827-bc8fa6c7e353", - "resource": { - "resourceType": "Claim", - "id": "0f9bc1f1-05bc-4172-8827-bc8fa6c7e353", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "created": "2019-01-07T04:51:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:930c7097-a336-4d01-9c6b-ff038011ffe1" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:22e25df5-de3c-45da-959d-24a25a8553d1" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1959d580-9164-4a51-84cf-a374c3b97d22" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:9d0668a8-2b39-4a87-8c13-0bbf20a09d04" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 5, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 505.09, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8055cd60-5b93-4bf1-a7c7-c2d0e0d65eea", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8055cd60-5b93-4bf1-a7c7-c2d0e0d65eea", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0f9bc1f1-05bc-4172-8827-bc8fa6c7e353" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2019-01-07T04:51:42-08:00", - "end": "2020-01-07T04:51:42-08:00" - }, - "created": "2019-01-07T04:51:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0f9bc1f1-05bc-4172-8827-bc8fa6c7e353" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a721f251-d45e-40c1-8da1-1edf0b8e92f1" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "servicedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2019-01-07T04:21:42-08:00", - "end": "2019-01-07T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 505.09, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 101.018, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 404.072, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 505.09, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 505.09, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 741.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896", - "resource": { - "resourceType": "Encounter", - "id": "c9dd0522-d214-44e6-84a4-30e2751f1896", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Mr. Emerson869 Prohaska837" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "individual": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "display": "Dr. Lillia547 Gerlach374" - } - } - ], - "period": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0a8e1b96-88a3-49cf-97d9-5316f8dad8ff", - "resource": { - "resourceType": "Observation", - "id": "0a8e1b96-88a3-49cf-97d9-5316f8dad8ff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 174.8, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2aabc75c-abf4-4453-9248-a0307713d1ae", - "resource": { - "resourceType": "Observation", - "id": "2aabc75c-abf4-4453-9248-a0307713d1ae", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1502e9d5-4125-42b5-9847-77319e2551a5", - "resource": { - "resourceType": "Observation", - "id": "1502e9d5-4125-42b5-9847-77319e2551a5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 91.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32c9d0d2-c8a5-4e3f-8755-50168d5dd802", - "resource": { - "resourceType": "Observation", - "id": "32c9d0d2-c8a5-4e3f-8755-50168d5dd802", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 29.95, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51bff806-dbe9-4816-ba5b-788986cd06cd", - "resource": { - "resourceType": "Observation", - "id": "51bff806-dbe9-4816-ba5b-788986cd06cd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 116, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2793e50e-36fa-471a-b94b-b2c07b48d352", - "resource": { - "resourceType": "Observation", - "id": "2793e50e-36fa-471a-b94b-b2c07b48d352", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 90, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2315a08e-491b-401f-831e-6343392eb258", - "resource": { - "resourceType": "Observation", - "id": "2315a08e-491b-401f-831e-6343392eb258", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc4daf50-6cfa-4650-9e75-8f333ecec4d1", - "resource": { - "resourceType": "Observation", - "id": "dc4daf50-6cfa-4650-9e75-8f333ecec4d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 96.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:27b36e11-a573-4105-80f7-4264fb24b297", - "resource": { - "resourceType": "Observation", - "id": "27b36e11-a573-4105-80f7-4264fb24b297", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 10.81, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8d02f82f-f29a-4a9e-8757-c1fed998352f", - "resource": { - "resourceType": "Observation", - "id": "8d02f82f-f29a-4a9e-8757-c1fed998352f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 1.12, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7455a716-3c97-4b44-818e-4202ef3cd1dd", - "resource": { - "resourceType": "Observation", - "id": "7455a716-3c97-4b44-818e-4202ef3cd1dd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 9.33, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", - "resource": { - "resourceType": "Observation", - "id": "fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 140.64, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", - "resource": { - "resourceType": "Observation", - "id": "1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.76, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d6fb2d0d-a442-472e-a7f0-691a968bf4f0", - "resource": { - "resourceType": "Observation", - "id": "d6fb2d0d-a442-472e-a7f0-691a968bf4f0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 105.18, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b709490-a933-427f-8580-1db7d10b95a1", - "resource": { - "resourceType": "Observation", - "id": "8b709490-a933-427f-8580-1db7d10b95a1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 26.06, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddc1d4aa-8010-43a1-a764-6d0385671973", - "resource": { - "resourceType": "Observation", - "id": "ddc1d4aa-8010-43a1-a764-6d0385671973", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.7327, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d70bb003-0c32-4d7a-af9f-3374f3db3da1", - "resource": { - "resourceType": "Observation", - "id": "d70bb003-0c32-4d7a-af9f-3374f3db3da1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 4.1641, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67c95ca7-4af0-4c5f-aea0-a4a943297f65", - "resource": { - "resourceType": "Observation", - "id": "67c95ca7-4af0-4c5f-aea0-a4a943297f65", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 13.07, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:755d1096-d7b0-430a-bce1-ea660114a66e", - "resource": { - "resourceType": "Observation", - "id": "755d1096-d7b0-430a-bce1-ea660114a66e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 39.996, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", - "resource": { - "resourceType": "Observation", - "id": "ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 89.485, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e6af9c0-c3fe-4eaf-b609-710737833383", - "resource": { - "resourceType": "Observation", - "id": "1e6af9c0-c3fe-4eaf-b609-710737833383", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 31.815, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a628ef78-13b8-4279-9909-efe174ac51b3", - "resource": { - "resourceType": "Observation", - "id": "a628ef78-13b8-4279-9909-efe174ac51b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 34.573, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70d77739-25c0-4ce1-8196-55e4feea73d2", - "resource": { - "resourceType": "Observation", - "id": "70d77739-25c0-4ce1-8196-55e4feea73d2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 45.279, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", - "resource": { - "resourceType": "Observation", - "id": "ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 183.14, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b81eca3-6be0-462b-9650-5152a567bf2a", - "resource": { - "resourceType": "Observation", - "id": "1b81eca3-6be0-462b-9650-5152a567bf2a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 239.92, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8ae6ef1d-78ab-4655-9fba-2f4954b53f48", - "resource": { - "resourceType": "Observation", - "id": "8ae6ef1d-78ab-4655-9fba-2f4954b53f48", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 11.424, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bfa3ab3a-3ac8-4c8c-b061-b831c8980c3b", - "resource": { - "resourceType": "Observation", - "id": "bfa3ab3a-3ac8-4c8c-b061-b831c8980c3b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31e62da5-29f6-4120-908c-2d55a557bb04", - "resource": { - "resourceType": "Observation", - "id": "31e62da5-29f6-4120-908c-2d55a557bb04", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "valueQuantity": { - "value": 5.87, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d3d6b965-7193-4a81-93ae-1bc4539f6c06", - "resource": { - "resourceType": "Procedure", - "id": "d3d6b965-7193-4a81-93ae-1bc4539f6c06", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "performedPeriod": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:36:42-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a2658308-c442-4a5a-84ca-682a542f56aa", - "resource": { - "resourceType": "Immunization", - "id": "a2658308-c442-4a5a-84ca-682a542f56aa", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "occurrenceDateTime": "2020-01-13T04:21:42-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:9382b897-111d-4662-a591-e8b130d21816", - "resource": { - "resourceType": "DiagnosticReport", - "id": "9382b897-111d-4662-a591-e8b130d21816", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:dc4daf50-6cfa-4650-9e75-8f333ecec4d1", - "display": "Glucose" - }, - { - "reference": "urn:uuid:27b36e11-a573-4105-80f7-4264fb24b297", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:8d02f82f-f29a-4a9e-8757-c1fed998352f", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:7455a716-3c97-4b44-818e-4202ef3cd1dd", - "display": "Calcium" - }, - { - "reference": "urn:uuid:fb9b4f44-dde3-47e3-9fe0-f1bf3293809e", - "display": "Sodium" - }, - { - "reference": "urn:uuid:1174b6fd-59d1-41d7-8b0e-d42c9f3d32ab", - "display": "Potassium" - }, - { - "reference": "urn:uuid:d6fb2d0d-a442-472e-a7f0-691a968bf4f0", - "display": "Chloride" - }, - { - "reference": "urn:uuid:8b709490-a933-427f-8580-1db7d10b95a1", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:66428479-9d74-4926-a182-c74b1a841a0f", - "resource": { - "resourceType": "DiagnosticReport", - "id": "66428479-9d74-4926-a182-c74b1a841a0f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "encounter": { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - }, - "effectiveDateTime": "2020-01-13T04:21:42-08:00", - "issued": "2020-01-13T04:21:42.840-08:00", - "result": [ - { - "reference": "urn:uuid:ddc1d4aa-8010-43a1-a764-6d0385671973", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:d70bb003-0c32-4d7a-af9f-3374f3db3da1", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:67c95ca7-4af0-4c5f-aea0-a4a943297f65", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:755d1096-d7b0-430a-bce1-ea660114a66e", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:ef89c9dd-36a2-4e2a-9fce-460bcad3d9a7", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:1e6af9c0-c3fe-4eaf-b609-710737833383", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:a628ef78-13b8-4279-9909-efe174ac51b3", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:70d77739-25c0-4ce1-8196-55e4feea73d2", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:ad8853d1-7ece-4bf7-8950-b2bdb079e8d1", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1b81eca3-6be0-462b-9650-5152a567bf2a", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:8ae6ef1d-78ab-4655-9fba-2f4954b53f48", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:a0c609cd-ca62-45d8-8051-54c4316c4a8a", - "resource": { - "resourceType": "Claim", - "id": "a0c609cd-ca62-45d8-8051-54c4316c4a8a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6", - "display": "Emerson869 Prohaska837" - }, - "billablePeriod": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "created": "2020-01-13T04:51:42-08:00", - "provider": { - "reference": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "display": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a2658308-c442-4a5a-84ca-682a542f56aa" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:d3d6b965-7193-4a81-93ae-1bc4539f6c06" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 991.58, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bbe8a13e-8315-4a7f-b51e-bd4884fd1d38", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "bbe8a13e-8315-4a7f-b51e-bd4884fd1d38", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "requester": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "performer": [ - { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a0c609cd-ca62-45d8-8051-54c4316c4a8a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:d89e3bd8-4d0f-4735-8d16-d0f646a9dcd6" - }, - "billablePeriod": { - "start": "2020-01-13T04:51:42-08:00", - "end": "2021-01-13T04:51:42-08:00" - }, - "created": "2020-01-13T04:51:42-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a0c609cd-ca62-45d8-8051-54c4316c4a8a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c9dd0522-d214-44e6-84a4-30e2751f1896" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2020-01-13T04:21:42-08:00", - "end": "2020-01-13T04:51:42-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 991.58, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 198.31600000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 793.2640000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 991.58, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 991.58, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 905.6800000000002, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json deleted file mode 100644 index 901fdf2e3177..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Lorette239_Marvin195_af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b.json +++ /dev/null @@ -1,31841 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "resource": { - "resourceType": "Patient", - "id": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7730420515391289066 Population seed: 1586368870505
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Magnolia736 Weber641" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", - "valueAddress": { - "city": "Concord", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 5.141867221161322 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 53.85813277883868 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "SS", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-30-3400" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99966155" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "PPN", - "display": "Passport Number" - } - ], - "text": "Passport Number" - }, - "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", - "value": "X73794696X" - } - ], - "name": [ - { - "use": "official", - "family": "Marvin195", - "given": [ - "Lorette239" - ], - "prefix": [ - "Mrs." - ] - }, - { - "use": "maiden", - "family": "Zemlak964", - "given": [ - "Lorette239" - ], - "prefix": [ - "Mrs." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-259-1705", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1960-10-02", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.378055340679865 - }, - { - "url": "longitude", - "valueDecimal": -71.01059430318017 - } - ] - } - ], - "line": [ - "755 Von Mission" - ], - "city": "Boston", - "state": "Massachusetts", - "postalCode": "02113", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", - "code": "M", - "display": "M" - } - ], - "text": "M" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "resource": { - "resourceType": "Organization", - "id": "76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "76a80350-916e-3fc9-8fed-2c8c4b36c42f" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC.", - "telecom": [ - { - "system": "phone", - "value": "617-912-7900" - } - ], - "address": [ - { - "line": [ - "301 BROADWAY" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "02150-2807", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "resource": { - "resourceType": "Practitioner", - "id": "0390078a-1894-335b-9136-5f518ee1dbb6", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999931799" - } - ], - "active": true, - "name": [ - { - "family": "Koelpin146", - "given": [ - "Cheryle584" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Cheryle584.Koelpin146@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "301 BROADWAY" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "02150-2807", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7", - "resource": { - "resourceType": "Encounter", - "id": "f2ae7ab9-8abe-424a-a52b-72d3603542b7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1982-12-05T22:43:39-08:00", - "end": "1982-12-05T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "1982-12-05T22:43:39-08:00", - "end": "1982-12-05T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c", - "resource": { - "resourceType": "Condition", - "id": "788dd653-78c0-4922-a4ec-fcd8f720e60c", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" - }, - "onsetDateTime": "1982-12-05T22:43:39-08:00", - "recordedDate": "1982-12-05T22:43:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:bf763ed9-87a4-4aa4-853e-7aa252cbec05", - "resource": { - "resourceType": "CareTeam", - "id": "bf763ed9-87a4-4aa4-853e-7aa252cbec05", - "status": "active", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" - }, - "period": { - "start": "1982-12-05T22:43:39-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:927b39fe-945e-4b26-961e-c856bcadc2fb", - "resource": { - "resourceType": "CarePlan", - "id": "927b39fe-945e-4b26-961e-c856bcadc2fb", - "text": { - "status": "generated", - "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698360004", - "display": "Diabetes self management plan" - } - ], - "text": "Diabetes self management plan" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" - }, - "period": { - "start": "1982-12-05T22:43:39-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:bf763ed9-87a4-4aa4-853e-7aa252cbec05" - } - ], - "addresses": [ - { - "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "160670007", - "display": "Diabetic diet" - } - ], - "text": "Diabetic diet" - }, - "status": "in-progress", - "location": { - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "location": { - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:6e42015f-6507-409b-a2ba-08cba73365f4", - "resource": { - "resourceType": "Claim", - "id": "6e42015f-6507-409b-a2ba-08cba73365f4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "1982-12-05T22:43:39-08:00", - "end": "1982-12-05T23:13:39-08:00" - }, - "created": "1982-12-05T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cd3adb27-2387-401c-b16e-152e4a914e40", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cd3adb27-2387-401c-b16e-152e4a914e40", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6e42015f-6507-409b-a2ba-08cba73365f4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "1982-12-05T23:13:39-08:00", - "end": "1983-12-05T23:13:39-08:00" - }, - "created": "1982-12-05T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:6e42015f-6507-409b-a2ba-08cba73365f4" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:788dd653-78c0-4922-a4ec-fcd8f720e60c" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1982-12-05T22:43:39-08:00", - "end": "1982-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f2ae7ab9-8abe-424a-a52b-72d3603542b7" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "servicedPeriod": { - "start": "1982-12-05T22:43:39-08:00", - "end": "1982-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14", - "resource": { - "resourceType": "Encounter", - "id": "5b0e5a93-b152-4a67-8cfc-05fa6ef49a14", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1985-12-08T22:43:39-08:00", - "end": "1985-12-08T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "1985-12-08T22:43:39-08:00", - "end": "1985-12-08T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba", - "resource": { - "resourceType": "Condition", - "id": "8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" - }, - "onsetDateTime": "1985-12-08T22:43:39-08:00", - "recordedDate": "1985-12-08T22:43:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:444b38a1-5335-42ad-a80d-6a02e9fe245f", - "resource": { - "resourceType": "Claim", - "id": "444b38a1-5335-42ad-a80d-6a02e9fe245f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "1985-12-08T22:43:39-08:00", - "end": "1985-12-08T23:13:39-08:00" - }, - "created": "1985-12-08T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Aetna" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:383d050d-44e4-42a7-98fd-969541d92aa0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "383d050d-44e4-42a7-98fd-969541d92aa0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Aetna" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Aetna" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "444b38a1-5335-42ad-a80d-6a02e9fe245f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "1985-12-08T23:13:39-08:00", - "end": "1986-12-08T23:13:39-08:00" - }, - "created": "1985-12-08T23:13:39-08:00", - "insurer": { - "display": "Aetna" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:444b38a1-5335-42ad-a80d-6a02e9fe245f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8ea60d56-7d67-4cc3-b4b6-e88f170bc8ba" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Aetna" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1985-12-08T22:43:39-08:00", - "end": "1985-12-08T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:5b0e5a93-b152-4a67-8cfc-05fa6ef49a14" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "271737000", - "display": "Anemia (disorder)" - } - ], - "text": "Anemia (disorder)" - }, - "servicedPeriod": { - "start": "1985-12-08T22:43:39-08:00", - "end": "1985-12-08T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "resource": { - "resourceType": "Organization", - "id": "d733d4a9-080d-3593-b910-2366e652b7ea", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d733d4a9-080d-3593-b910-2366e652b7ea" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6179837000" - } - ], - "address": [ - { - "line": [ - "1153 CENTRE STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02130", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "resource": { - "resourceType": "Practitioner", - "id": "3a576225-3a87-3704-a672-e1c8bc03389f", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999519" - } - ], - "active": true, - "name": [ - { - "family": "Howell947", - "given": [ - "Max124" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Max124.Howell947@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "1153 CENTRE STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02130", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6", - "resource": { - "resourceType": "Encounter", - "id": "52ff6ac0-6eaf-45ec-869a-60a846973ab6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1986-10-26T22:43:39-08:00", - "end": "1986-10-26T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "display": "Dr. Max124 Howell947" - } - } - ], - "period": { - "start": "1986-10-26T22:43:39-08:00", - "end": "1986-10-26T23:28:39-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6", - "resource": { - "resourceType": "Condition", - "id": "c2003678-0785-4395-bdb1-0311e15128c6", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - ], - "text": "Miscarriage in first trimester" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" - }, - "onsetDateTime": "1986-10-26T22:43:39-08:00", - "recordedDate": "1986-10-26T22:43:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45", - "resource": { - "resourceType": "Claim", - "id": "6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "1986-10-26T22:43:39-08:00", - "end": "1986-10-26T23:28:39-08:00" - }, - "created": "1986-10-26T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - }, - "encounter": [ - { - "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - ], - "text": "Miscarriage in first trimester" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cd9388fe-54d7-4057-8ade-975d7457b917", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cd9388fe-54d7-4057-8ade-975d7457b917", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "performer": [ - { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "1986-10-26T23:28:39-08:00", - "end": "1987-10-26T23:28:39-08:00" - }, - "created": "1986-10-26T23:28:39-08:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:6bbd26b9-9a00-4b7e-b9c8-83e1ddae7f45" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:c2003678-0785-4395-bdb1-0311e15128c6" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - }, - "servicedPeriod": { - "start": "1986-10-26T22:43:39-08:00", - "end": "1986-10-26T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:52ff6ac0-6eaf-45ec-869a-60a846973ab6" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "19169002", - "display": "Miscarriage in first trimester" - } - ], - "text": "Miscarriage in first trimester" - }, - "servicedPeriod": { - "start": "1986-10-26T22:43:39-08:00", - "end": "1986-10-26T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424", - "resource": { - "resourceType": "Encounter", - "id": "2ccc9f08-6dee-48fe-9b62-a7aed300e424", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1997-12-21T22:43:39-08:00", - "end": "1997-12-21T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "1997-12-21T22:43:39-08:00", - "end": "1997-12-21T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "resource": { - "resourceType": "Condition", - "id": "f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "87433001", - "display": "Pulmonary emphysema (disorder)" - } - ], - "text": "Pulmonary emphysema (disorder)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" - }, - "onsetDateTime": "1997-12-21T22:43:39-08:00", - "recordedDate": "1997-12-21T22:43:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:461a5652-6d82-4e33-b0a6-44229bea93ed", - "resource": { - "resourceType": "CareTeam", - "id": "461a5652-6d82-4e33-b0a6-44229bea93ed", - "status": "active", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" - }, - "period": { - "start": "1997-12-21T22:43:39-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "87433001", - "display": "Pulmonary emphysema (disorder)" - } - ], - "text": "Pulmonary emphysema (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:ab5480fe-a33d-4188-9a89-f01ce663f0fd", - "resource": { - "resourceType": "CarePlan", - "id": "ab5480fe-a33d-4188-9a89-f01ce663f0fd", - "text": { - "status": "generated", - "div": "
    Care Plan for Chronic obstructive pulmonary disease clinical management plan.
    Activities:
    • Chronic obstructive pulmonary disease clinical management plan
    • Chronic obstructive pulmonary disease clinical management plan

    Care plan is meant to treat Pulmonary emphysema (disorder).
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "412776001", - "display": "Chronic obstructive pulmonary disease clinical management plan" - } - ], - "text": "Chronic obstructive pulmonary disease clinical management plan" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" - }, - "period": { - "start": "1997-12-21T22:43:39-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:461a5652-6d82-4e33-b0a6-44229bea93ed" - } - ], - "addresses": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "location": { - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15081005", - "display": "Pulmonary rehabilitation (regime/therapy)" - } - ], - "text": "Pulmonary rehabilitation (regime/therapy)" - }, - "status": "in-progress", - "location": { - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:7341a535-24a6-4871-8750-b71c03f22e4d", - "resource": { - "resourceType": "Claim", - "id": "7341a535-24a6-4871-8750-b71c03f22e4d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "1997-12-21T22:43:39-08:00", - "end": "1997-12-21T23:13:39-08:00" - }, - "created": "1997-12-21T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "87433001", - "display": "Pulmonary emphysema (disorder)" - } - ], - "text": "Pulmonary emphysema (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f07c5217-22d2-44cc-b327-07987b795cfa", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f07c5217-22d2-44cc-b327-07987b795cfa", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7341a535-24a6-4871-8750-b71c03f22e4d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "1997-12-21T23:13:39-08:00", - "end": "1998-12-21T23:13:39-08:00" - }, - "created": "1997-12-21T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7341a535-24a6-4871-8750-b71c03f22e4d" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1997-12-21T22:43:39-08:00", - "end": "1997-12-21T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2ccc9f08-6dee-48fe-9b62-a7aed300e424" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "87433001", - "display": "Pulmonary emphysema (disorder)" - } - ], - "text": "Pulmonary emphysema (disorder)" - }, - "servicedPeriod": { - "start": "1997-12-21T22:43:39-08:00", - "end": "1997-12-21T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0", - "resource": { - "resourceType": "Encounter", - "id": "35491525-a5e9-4f41-bfad-d92a0933e8d0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4a454ca9-849d-4b11-b0b8-636211ec2dbe", - "resource": { - "resourceType": "MedicationRequest", - "id": "4a454ca9-849d-4b11-b0b8-636211ec2dbe", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - }, - "authoredOn": "2000-10-08T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ec0f9c39-9a97-4230-9716-c51212ec8458", - "resource": { - "resourceType": "Claim", - "id": "ec0f9c39-9a97-4230-9716-c51212ec8458", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "created": "2000-10-09T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:4a454ca9-849d-4b11-b0b8-636211ec2dbe" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - } - ] - } - ], - "total": { - "value": 47.02, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cd0998fb-703c-4227-9aee-55c95ba6c0eb", - "resource": { - "resourceType": "MedicationRequest", - "id": "cd0998fb-703c-4227-9aee-55c95ba6c0eb", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - }, - "authoredOn": "2000-10-08T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ca39d049-e07a-421b-9afb-3b15b612cdbc", - "resource": { - "resourceType": "Claim", - "id": "ca39d049-e07a-421b-9afb-3b15b612cdbc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "created": "2000-10-09T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:cd0998fb-703c-4227-9aee-55c95ba6c0eb" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - } - ] - } - ], - "total": { - "value": 39.99, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ce915626-94c5-463f-bd62-e8b72a7c6cbe", - "resource": { - "resourceType": "Claim", - "id": "ce915626-94c5-463f-bd62-e8b72a7c6cbe", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "created": "2000-10-09T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:068b8034-f184-4029-896c-85706cff3b92", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "068b8034-f184-4029-896c-85706cff3b92", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ce915626-94c5-463f-bd62-e8b72a7c6cbe" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2000-10-09T00:13:39-07:00", - "end": "2001-10-09T00:13:39-07:00" - }, - "created": "2000-10-09T00:13:39-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ce915626-94c5-463f-bd62-e8b72a7c6cbe" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2000-10-08T23:43:39-07:00", - "end": "2000-10-09T00:13:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:35491525-a5e9-4f41-bfad-d92a0933e8d0" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96", - "resource": { - "resourceType": "Encounter", - "id": "95112402-82bc-4ab9-912f-607f28d5ee96", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2001-10-14T23:43:39-07:00", - "end": "2001-10-15T00:13:39-07:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2001-10-14T23:43:39-07:00", - "end": "2001-10-15T00:13:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6e25a07d-403c-4270-b798-cd9b40f29e0c", - "resource": { - "resourceType": "MedicationRequest", - "id": "6e25a07d-403c-4270-b798-cd9b40f29e0c", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" - }, - "authoredOn": "2001-10-14T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:6706dbc8-36af-490e-b04e-4146f04e7bb1", - "resource": { - "resourceType": "Claim", - "id": "6706dbc8-36af-490e-b04e-4146f04e7bb1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2001-10-14T23:43:39-07:00", - "end": "2001-10-15T00:13:39-07:00" - }, - "created": "2001-10-15T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:6e25a07d-403c-4270-b798-cd9b40f29e0c" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" - } - ] - } - ], - "total": { - "value": 10.29, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4e26781a-79c9-4b4b-bb04-0f2ba2e359e9", - "resource": { - "resourceType": "Claim", - "id": "4e26781a-79c9-4b4b-bb04-0f2ba2e359e9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2001-10-14T23:43:39-07:00", - "end": "2001-10-15T00:13:39-07:00" - }, - "created": "2001-10-15T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:19a1350d-a290-4470-9a11-b056fa08c14f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "19a1350d-a290-4470-9a11-b056fa08c14f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4e26781a-79c9-4b4b-bb04-0f2ba2e359e9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2001-10-15T00:13:39-07:00", - "end": "2002-10-15T00:13:39-07:00" - }, - "created": "2001-10-15T00:13:39-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4e26781a-79c9-4b4b-bb04-0f2ba2e359e9" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2001-10-14T23:43:39-07:00", - "end": "2001-10-15T00:13:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:95112402-82bc-4ab9-912f-607f28d5ee96" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c", - "resource": { - "resourceType": "Encounter", - "id": "4007e9d7-ca83-4a4b-ba98-e2115d096c0c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2002-10-20T23:43:39-07:00", - "end": "2002-10-21T00:13:39-07:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2002-10-20T23:43:39-07:00", - "end": "2002-10-21T00:13:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2d35ad83-68c9-48d8-a364-df62c353b26b", - "resource": { - "resourceType": "MedicationRequest", - "id": "2d35ad83-68c9-48d8-a364-df62c353b26b", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" - }, - "authoredOn": "2002-10-20T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:08b60435-7cb7-4ca4-a235-9cbed724a63b", - "resource": { - "resourceType": "Claim", - "id": "08b60435-7cb7-4ca4-a235-9cbed724a63b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2002-10-20T23:43:39-07:00", - "end": "2002-10-21T00:13:39-07:00" - }, - "created": "2002-10-21T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:2d35ad83-68c9-48d8-a364-df62c353b26b" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" - } - ] - } - ], - "total": { - "value": 19.73, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1835b212-674d-4caa-a2b4-cab25b255294", - "resource": { - "resourceType": "Claim", - "id": "1835b212-674d-4caa-a2b4-cab25b255294", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2002-10-20T23:43:39-07:00", - "end": "2002-10-21T00:13:39-07:00" - }, - "created": "2002-10-21T00:13:39-07:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ee7c7505-fd48-4804-a86a-ab23e5cf3efa", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ee7c7505-fd48-4804-a86a-ab23e5cf3efa", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1835b212-674d-4caa-a2b4-cab25b255294" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2002-10-21T00:13:39-07:00", - "end": "2003-10-21T00:13:39-07:00" - }, - "created": "2002-10-21T00:13:39-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:1835b212-674d-4caa-a2b4-cab25b255294" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2002-10-20T23:43:39-07:00", - "end": "2002-10-21T00:13:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4007e9d7-ca83-4a4b-ba98-e2115d096c0c" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242", - "resource": { - "resourceType": "Encounter", - "id": "4a967f12-9da3-4ade-87e2-a2a668cd5242", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2003-10-26T22:43:39-08:00", - "end": "2003-10-26T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2003-10-26T22:43:39-08:00", - "end": "2003-10-26T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a6520d80-4d23-453a-8ab0-fb6ecc05a179", - "resource": { - "resourceType": "MedicationRequest", - "id": "a6520d80-4d23-453a-8ab0-fb6ecc05a179", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" - }, - "authoredOn": "2003-10-26T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:09a033a7-7010-4f3f-8781-cf2619dd12bc", - "resource": { - "resourceType": "Claim", - "id": "09a033a7-7010-4f3f-8781-cf2619dd12bc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2003-10-26T22:43:39-08:00", - "end": "2003-10-26T23:13:39-08:00" - }, - "created": "2003-10-26T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:a6520d80-4d23-453a-8ab0-fb6ecc05a179" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" - } - ] - } - ], - "total": { - "value": 15.01, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ff57c60e-c6fa-4084-a102-9d1ef4715aaa", - "resource": { - "resourceType": "Claim", - "id": "ff57c60e-c6fa-4084-a102-9d1ef4715aaa", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2003-10-26T22:43:39-08:00", - "end": "2003-10-26T23:13:39-08:00" - }, - "created": "2003-10-26T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1d4f3c3e-27ab-46ea-98ca-f8d6aa9a89d3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1d4f3c3e-27ab-46ea-98ca-f8d6aa9a89d3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ff57c60e-c6fa-4084-a102-9d1ef4715aaa" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2003-10-26T23:13:39-08:00", - "end": "2004-10-26T23:13:39-07:00" - }, - "created": "2003-10-26T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ff57c60e-c6fa-4084-a102-9d1ef4715aaa" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2003-10-26T22:43:39-08:00", - "end": "2003-10-26T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4a967f12-9da3-4ade-87e2-a2a668cd5242" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "resource": { - "resourceType": "Organization", - "id": "44bef9d3-91c2-3005-93e0-ccf436348ff0", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "44bef9d3-91c2-3005-93e0-ccf436348ff0" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MASSACHUSETTS EYE AND EAR INFIRMARY -", - "telecom": [ - { - "system": "phone", - "value": "6175237900" - } - ], - "address": [ - { - "line": [ - "243 CHARLES STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "resource": { - "resourceType": "Practitioner", - "id": "fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999689" - } - ], - "active": true, - "name": [ - { - "family": "Mayert710", - "given": [ - "Dovie983" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Dovie983.Mayert710@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "243 CHARLES STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4", - "resource": { - "resourceType": "Encounter", - "id": "55ed12fc-16f8-4088-bb03-a0129f0606b4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2003-12-28T22:43:39-08:00", - "end": "2003-12-28T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - } - ], - "period": { - "start": "2003-12-28T22:43:39-08:00", - "end": "2003-12-28T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f83ae010-a644-4ecd-9eea-199e3452f2f3", - "resource": { - "resourceType": "MedicationRequest", - "id": "f83ae010-a644-4ecd-9eea-199e3452f2f3", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" - }, - "authoredOn": "2003-12-28T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:44fb369c-f31c-40a0-92b2-7b1ad39c5f4e", - "resource": { - "resourceType": "Claim", - "id": "44fb369c-f31c-40a0-92b2-7b1ad39c5f4e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2003-12-28T22:43:39-08:00", - "end": "2003-12-28T23:13:39-08:00" - }, - "created": "2003-12-28T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:f83ae010-a644-4ecd-9eea-199e3452f2f3" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" - } - ] - } - ], - "total": { - "value": 29.28, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d88517ec-bb58-4b93-8173-2f21f4e45b64", - "resource": { - "resourceType": "Claim", - "id": "d88517ec-bb58-4b93-8173-2f21f4e45b64", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2003-12-28T22:43:39-08:00", - "end": "2003-12-28T23:13:39-08:00" - }, - "created": "2003-12-28T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6c92aff6-8546-45e1-bfcd-5e9d0819af6e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6c92aff6-8546-45e1-bfcd-5e9d0819af6e", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "performer": [ - { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d88517ec-bb58-4b93-8173-2f21f4e45b64" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2003-12-28T23:13:39-08:00", - "end": "2004-12-28T23:13:39-08:00" - }, - "created": "2003-12-28T23:13:39-08:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:d88517ec-bb58-4b93-8173-2f21f4e45b64" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "2003-12-28T22:43:39-08:00", - "end": "2003-12-28T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:55ed12fc-16f8-4088-bb03-a0129f0606b4" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6", - "resource": { - "resourceType": "Encounter", - "id": "b691ff04-6df6-404b-8560-ea8adfdd64f6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2004-10-31T22:43:39-08:00", - "end": "2004-10-31T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2004-10-31T22:43:39-08:00", - "end": "2004-10-31T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6862a1eb-8d30-4b28-8e51-5f00397eaea4", - "resource": { - "resourceType": "MedicationRequest", - "id": "6862a1eb-8d30-4b28-8e51-5f00397eaea4", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" - }, - "authoredOn": "2004-10-31T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a509d9fd-9b8d-4ac8-b30d-42952b34ebb7", - "resource": { - "resourceType": "Claim", - "id": "a509d9fd-9b8d-4ac8-b30d-42952b34ebb7", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2004-10-31T22:43:39-08:00", - "end": "2004-10-31T23:13:39-08:00" - }, - "created": "2004-10-31T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:6862a1eb-8d30-4b28-8e51-5f00397eaea4" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" - } - ] - } - ], - "total": { - "value": 26.81, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a1f029a5-53ac-4d92-8e65-fc409aa0a4c5", - "resource": { - "resourceType": "Claim", - "id": "a1f029a5-53ac-4d92-8e65-fc409aa0a4c5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2004-10-31T22:43:39-08:00", - "end": "2004-10-31T23:13:39-08:00" - }, - "created": "2004-10-31T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3e555a99-17cb-4f42-b31f-f5a2ad3b4a4d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3e555a99-17cb-4f42-b31f-f5a2ad3b4a4d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a1f029a5-53ac-4d92-8e65-fc409aa0a4c5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2004-10-31T23:13:39-08:00", - "end": "2005-10-31T23:13:39-08:00" - }, - "created": "2004-10-31T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a1f029a5-53ac-4d92-8e65-fc409aa0a4c5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2004-10-31T22:43:39-08:00", - "end": "2004-10-31T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b691ff04-6df6-404b-8560-ea8adfdd64f6" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26", - "resource": { - "resourceType": "Encounter", - "id": "80ca511b-b8fb-4219-b33b-89d241e69d26", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2005-11-06T22:43:39-08:00", - "end": "2005-11-06T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2005-11-06T22:43:39-08:00", - "end": "2005-11-06T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:839732db-f78d-467c-a29c-3e7b6d9cf9be", - "resource": { - "resourceType": "MedicationRequest", - "id": "839732db-f78d-467c-a29c-3e7b6d9cf9be", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" - }, - "authoredOn": "2005-11-06T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:725d52e5-4a61-44b1-8009-1c09d7afe0db", - "resource": { - "resourceType": "Claim", - "id": "725d52e5-4a61-44b1-8009-1c09d7afe0db", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2005-11-06T22:43:39-08:00", - "end": "2005-11-06T23:13:39-08:00" - }, - "created": "2005-11-06T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:839732db-f78d-467c-a29c-3e7b6d9cf9be" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" - } - ] - } - ], - "total": { - "value": 25.32, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:56fc389f-cb27-4a04-8c74-c345ea700456", - "resource": { - "resourceType": "Claim", - "id": "56fc389f-cb27-4a04-8c74-c345ea700456", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2005-11-06T22:43:39-08:00", - "end": "2005-11-06T23:13:39-08:00" - }, - "created": "2005-11-06T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a39f4ff8-705b-488c-88c9-3a69a138908a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a39f4ff8-705b-488c-88c9-3a69a138908a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "56fc389f-cb27-4a04-8c74-c345ea700456" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2005-11-06T23:13:39-08:00", - "end": "2006-11-06T23:13:39-08:00" - }, - "created": "2005-11-06T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:56fc389f-cb27-4a04-8c74-c345ea700456" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2005-11-06T22:43:39-08:00", - "end": "2005-11-06T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:80ca511b-b8fb-4219-b33b-89d241e69d26" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280", - "resource": { - "resourceType": "Encounter", - "id": "42d5eaf7-1718-4646-884e-541c2a45b280", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2006-11-12T22:43:39-08:00", - "end": "2006-11-12T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2006-11-12T22:43:39-08:00", - "end": "2006-11-12T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:90e50881-0517-4120-b7bf-d71e3a9d5332", - "resource": { - "resourceType": "MedicationRequest", - "id": "90e50881-0517-4120-b7bf-d71e3a9d5332", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" - }, - "authoredOn": "2006-11-12T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:573f7735-70da-4176-9950-95cf26936807", - "resource": { - "resourceType": "Claim", - "id": "573f7735-70da-4176-9950-95cf26936807", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2006-11-12T22:43:39-08:00", - "end": "2006-11-12T23:28:39-08:00" - }, - "created": "2006-11-12T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:90e50881-0517-4120-b7bf-d71e3a9d5332" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" - } - ] - } - ], - "total": { - "value": 21.18, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7fce27cc-7d95-4ea5-b103-80dbdf37b26a", - "resource": { - "resourceType": "Claim", - "id": "7fce27cc-7d95-4ea5-b103-80dbdf37b26a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2006-11-12T22:43:39-08:00", - "end": "2006-11-12T23:28:39-08:00" - }, - "created": "2006-11-12T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6a7d6334-f9d8-4213-98a9-1a806966b62d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6a7d6334-f9d8-4213-98a9-1a806966b62d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7fce27cc-7d95-4ea5-b103-80dbdf37b26a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2006-11-12T23:28:39-08:00", - "end": "2007-11-12T23:28:39-08:00" - }, - "created": "2006-11-12T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7fce27cc-7d95-4ea5-b103-80dbdf37b26a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2006-11-12T22:43:39-08:00", - "end": "2006-11-12T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:42d5eaf7-1718-4646-884e-541c2a45b280" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507", - "resource": { - "resourceType": "Encounter", - "id": "a5a881df-72cc-455d-980c-4a0f81459507", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2007-11-18T22:43:39-08:00", - "end": "2007-11-18T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2007-11-18T22:43:39-08:00", - "end": "2007-11-18T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:25cf1693-7aca-4df1-9680-aaa1dd71843f", - "resource": { - "resourceType": "MedicationRequest", - "id": "25cf1693-7aca-4df1-9680-aaa1dd71843f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" - }, - "authoredOn": "2007-11-18T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:01e64044-0466-4df2-a0d1-ae8347935aea", - "resource": { - "resourceType": "Claim", - "id": "01e64044-0466-4df2-a0d1-ae8347935aea", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2007-11-18T22:43:39-08:00", - "end": "2007-11-18T23:13:39-08:00" - }, - "created": "2007-11-18T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:25cf1693-7aca-4df1-9680-aaa1dd71843f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" - } - ] - } - ], - "total": { - "value": 27.87, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:272d2b52-bbff-4217-ae09-26e4d49640d0", - "resource": { - "resourceType": "Claim", - "id": "272d2b52-bbff-4217-ae09-26e4d49640d0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2007-11-18T22:43:39-08:00", - "end": "2007-11-18T23:13:39-08:00" - }, - "created": "2007-11-18T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3e0605f6-f4f5-4f5e-a54e-a1e1244c5665", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3e0605f6-f4f5-4f5e-a54e-a1e1244c5665", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "272d2b52-bbff-4217-ae09-26e4d49640d0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2007-11-18T23:13:39-08:00", - "end": "2008-11-18T23:13:39-08:00" - }, - "created": "2007-11-18T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:272d2b52-bbff-4217-ae09-26e4d49640d0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2007-11-18T22:43:39-08:00", - "end": "2007-11-18T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a5a881df-72cc-455d-980c-4a0f81459507" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f", - "resource": { - "resourceType": "Encounter", - "id": "35ffd429-2918-4891-8b60-6daf8ffe0b8f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2008-11-23T22:43:39-08:00", - "end": "2008-11-23T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2008-11-23T22:43:39-08:00", - "end": "2008-11-23T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:36b60b89-0a3f-42ae-9165-158611b69849", - "resource": { - "resourceType": "MedicationRequest", - "id": "36b60b89-0a3f-42ae-9165-158611b69849", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" - }, - "authoredOn": "2008-11-23T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:4abf2798-93b8-411c-947b-40b3bd84fcf2", - "resource": { - "resourceType": "Claim", - "id": "4abf2798-93b8-411c-947b-40b3bd84fcf2", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2008-11-23T22:43:39-08:00", - "end": "2008-11-23T23:28:39-08:00" - }, - "created": "2008-11-23T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:36b60b89-0a3f-42ae-9165-158611b69849" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" - } - ] - } - ], - "total": { - "value": 19.67, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4e37441d-055c-4244-a285-456f0ca248d5", - "resource": { - "resourceType": "Claim", - "id": "4e37441d-055c-4244-a285-456f0ca248d5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2008-11-23T22:43:39-08:00", - "end": "2008-11-23T23:28:39-08:00" - }, - "created": "2008-11-23T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:42e4f0c9-5d77-4a4d-8803-8dcb0a89a9b3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "42e4f0c9-5d77-4a4d-8803-8dcb0a89a9b3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4e37441d-055c-4244-a285-456f0ca248d5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2008-11-23T23:28:39-08:00", - "end": "2009-11-23T23:28:39-08:00" - }, - "created": "2008-11-23T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4e37441d-055c-4244-a285-456f0ca248d5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2008-11-23T22:43:39-08:00", - "end": "2008-11-23T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:35ffd429-2918-4891-8b60-6daf8ffe0b8f" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a", - "resource": { - "resourceType": "Encounter", - "id": "92136260-8183-4568-aef5-a720f0a63b5a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2009-06-23T23:43:39-07:00", - "end": "2009-06-24T00:23:39-07:00" - }, - "individual": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - } - ], - "period": { - "start": "2009-06-23T23:43:39-07:00", - "end": "2009-06-24T00:23:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5cd2892a-869d-4830-bd0c-953cd2d617f2", - "resource": { - "resourceType": "MedicationRequest", - "id": "5cd2892a-869d-4830-bd0c-953cd2d617f2", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "389221", - "display": "Etonogestrel 68 MG Drug Implant" - } - ], - "text": "Etonogestrel 68 MG Drug Implant" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" - }, - "authoredOn": "2009-06-23T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:fb52d101-c808-4709-89b7-2f1f00552ecb", - "resource": { - "resourceType": "Claim", - "id": "fb52d101-c808-4709-89b7-2f1f00552ecb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2009-06-23T23:43:39-07:00", - "end": "2009-06-24T00:23:39-07:00" - }, - "created": "2009-06-24T00:23:39-07:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:5cd2892a-869d-4830-bd0c-953cd2d617f2" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "encounter": [ - { - "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" - } - ] - } - ], - "total": { - "value": 652.94, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e0efce36-aa03-4aa6-99de-935b709f57be", - "resource": { - "resourceType": "Claim", - "id": "e0efce36-aa03-4aa6-99de-935b709f57be", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2009-06-23T23:43:39-07:00", - "end": "2009-06-24T00:23:39-07:00" - }, - "created": "2009-06-24T00:23:39-07:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "encounter": [ - { - "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6822102c-86b2-45de-b764-980f29bdab65", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6822102c-86b2-45de-b764-980f29bdab65", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "performer": [ - { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e0efce36-aa03-4aa6-99de-935b709f57be" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2009-06-24T00:23:39-07:00", - "end": "2010-06-24T00:23:39-07:00" - }, - "created": "2009-06-24T00:23:39-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e0efce36-aa03-4aa6-99de-935b709f57be" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "servicedPeriod": { - "start": "2009-06-23T23:43:39-07:00", - "end": "2009-06-24T00:23:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:92136260-8183-4568-aef5-a720f0a63b5a" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54", - "resource": { - "resourceType": "Encounter", - "id": "ba1f1989-030a-479c-9738-68fdf9b6aa54", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2009-11-29T22:43:39-08:00", - "end": "2009-11-29T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2009-11-29T22:43:39-08:00", - "end": "2009-11-29T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b09fd23e-17cd-46f0-80bc-cf6bd818c6d4", - "resource": { - "resourceType": "MedicationRequest", - "id": "b09fd23e-17cd-46f0-80bc-cf6bd818c6d4", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" - }, - "authoredOn": "2009-11-29T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:9b847e62-0da5-4586-862d-d747fc6eb3cc", - "resource": { - "resourceType": "Claim", - "id": "9b847e62-0da5-4586-862d-d747fc6eb3cc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2009-11-29T22:43:39-08:00", - "end": "2009-11-29T23:13:39-08:00" - }, - "created": "2009-11-29T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:b09fd23e-17cd-46f0-80bc-cf6bd818c6d4" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" - } - ] - } - ], - "total": { - "value": 23.23, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b0d0b831-8639-468c-b7b3-30bccc9b30f1", - "resource": { - "resourceType": "Claim", - "id": "b0d0b831-8639-468c-b7b3-30bccc9b30f1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2009-11-29T22:43:39-08:00", - "end": "2009-11-29T23:13:39-08:00" - }, - "created": "2009-11-29T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fab05b3d-4335-40a7-b497-513ba32928bf", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "fab05b3d-4335-40a7-b497-513ba32928bf", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b0d0b831-8639-468c-b7b3-30bccc9b30f1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2009-11-29T23:13:39-08:00", - "end": "2010-11-29T23:13:39-08:00" - }, - "created": "2009-11-29T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b0d0b831-8639-468c-b7b3-30bccc9b30f1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2009-11-29T22:43:39-08:00", - "end": "2009-11-29T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ba1f1989-030a-479c-9738-68fdf9b6aa54" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d", - "resource": { - "resourceType": "Encounter", - "id": "3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2010-06-18T23:43:39-07:00", - "end": "2010-06-18T23:58:39-07:00" - }, - "individual": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - } - ], - "period": { - "start": "2010-06-18T23:43:39-07:00", - "end": "2010-06-18T23:58:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3e216f89-3bae-41c4-87d2-e5e5fdb2cc28", - "resource": { - "resourceType": "MedicationRequest", - "id": "3e216f89-3bae-41c4-87d2-e5e5fdb2cc28", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "749762", - "display": "Seasonique 91 Day Pack" - } - ], - "text": "Seasonique 91 Day Pack" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" - }, - "authoredOn": "2010-06-18T23:43:39-07:00", - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0675548c-4bab-4e19-8aa2-730039a5d443", - "resource": { - "resourceType": "Claim", - "id": "0675548c-4bab-4e19-8aa2-730039a5d443", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2010-06-18T23:43:39-07:00", - "end": "2010-06-18T23:58:39-07:00" - }, - "created": "2010-06-18T23:58:39-07:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:3e216f89-3bae-41c4-87d2-e5e5fdb2cc28" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "encounter": [ - { - "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" - } - ] - } - ], - "total": { - "value": 31.88, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d27c0519-49cc-47ea-b498-32ef083873b9", - "resource": { - "resourceType": "Claim", - "id": "d27c0519-49cc-47ea-b498-32ef083873b9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2010-06-18T23:43:39-07:00", - "end": "2010-06-18T23:58:39-07:00" - }, - "created": "2010-06-18T23:58:39-07:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "encounter": [ - { - "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:64536aa6-cbfe-4bbc-9158-7727b725ce88", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "64536aa6-cbfe-4bbc-9158-7727b725ce88", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "performer": [ - { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d27c0519-49cc-47ea-b498-32ef083873b9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2010-06-18T23:58:39-07:00", - "end": "2011-06-18T23:58:39-07:00" - }, - "created": "2010-06-18T23:58:39-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:d27c0519-49cc-47ea-b498-32ef083873b9" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - }, - "servicedPeriod": { - "start": "2010-06-18T23:43:39-07:00", - "end": "2010-06-18T23:58:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3fab7b47-e1d4-437e-9f9c-5d33b4d82c1d" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5", - "resource": { - "resourceType": "Encounter", - "id": "0722e38b-0125-4f23-b29f-c619a66d4ef5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:31:39-07:00" - }, - "individual": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "display": "Dr. Max124 Howell947" - } - } - ], - "period": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:31:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b62a4fa4-c56f-4f8e-8270-7c17d118b636", - "resource": { - "resourceType": "Procedure", - "id": "b62a4fa4-c56f-4f8e-8270-7c17d118b636", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" - }, - "performedPeriod": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:16:39-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0", - "resource": { - "resourceType": "Claim", - "id": "1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:31:39-07:00" - }, - "created": "2010-10-04T00:31:39-07:00", - "provider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b62a4fa4-c56f-4f8e-8270-7c17d118b636" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 16816.16, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5946f3c3-56f0-492f-8859-cacefd6f53fb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5946f3c3-56f0-492f-8859-cacefd6f53fb", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "performer": [ - { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2010-10-04T00:31:39-07:00", - "end": "2011-10-04T00:31:39-07:00" - }, - "created": "2010-10-04T00:31:39-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:1e4ce9ad-e7f5-49d6-9d1c-a44ae6a942f0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:31:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0722e38b-0125-4f23-b29f-c619a66d4ef5" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2010-10-03T23:43:39-07:00", - "end": "2010-10-04T00:31:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 16816.16, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 3363.232, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 13452.928, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 16816.16, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 16816.16, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 13452.928, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524", - "resource": { - "resourceType": "Encounter", - "id": "b0e2b0cd-72e8-4b97-8dd2-6e388c01e524", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b884c0fd-e12f-49d0-a1ea-c8993d39dc32", - "resource": { - "resourceType": "Observation", - "id": "b884c0fd-e12f-49d0-a1ea-c8993d39dc32", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 67.379, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca0ceb79-a79f-4b69-8e94-9de20e94dc83", - "resource": { - "resourceType": "Observation", - "id": "ca0ceb79-a79f-4b69-8e94-9de20e94dc83", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9251ed3b-87b5-4b12-b152-7eb7871f87c5", - "resource": { - "resourceType": "Observation", - "id": "9251ed3b-87b5-4b12-b152-7eb7871f87c5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2a114b23-b9f8-4a43-bb3e-d865a9228ca0", - "resource": { - "resourceType": "Observation", - "id": "2a114b23-b9f8-4a43-bb3e-d865a9228ca0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 71.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8c9c5596-05a0-4f08-a7e9-59ac09237cb1", - "resource": { - "resourceType": "Observation", - "id": "8c9c5596-05a0-4f08-a7e9-59ac09237cb1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 27.52, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28aea52e-9ae7-4f24-a66f-836d63570dd4", - "resource": { - "resourceType": "Observation", - "id": "28aea52e-9ae7-4f24-a66f-836d63570dd4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 88, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 122, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5361305c-390f-4ef3-9093-0b08404cfdbd", - "resource": { - "resourceType": "Observation", - "id": "5361305c-390f-4ef3-9093-0b08404cfdbd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:10398e74-c8f3-4fa1-8938-37db79cf4581", - "resource": { - "resourceType": "Observation", - "id": "10398e74-c8f3-4fa1-8938-37db79cf4581", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7be083f3-8f42-40a8-9c36-1ca04255875f", - "resource": { - "resourceType": "Observation", - "id": "7be083f3-8f42-40a8-9c36-1ca04255875f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 86.79, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3777fdb4-4e26-441b-8760-59bf63aef0f0", - "resource": { - "resourceType": "Observation", - "id": "3777fdb4-4e26-441b-8760-59bf63aef0f0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a04df17d-795d-4971-b74d-605058b14505", - "resource": { - "resourceType": "Observation", - "id": "a04df17d-795d-4971-b74d-605058b14505", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29bc2948-0bed-40b6-9ac2-22e6129eef77", - "resource": { - "resourceType": "Observation", - "id": "29bc2948-0bed-40b6-9ac2-22e6129eef77", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 8.5, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:81f6ca85-5291-4753-8246-86bdfa0d8779", - "resource": { - "resourceType": "Observation", - "id": "81f6ca85-5291-4753-8246-86bdfa0d8779", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 139.14, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e251d63-421d-406e-8cfd-3d6c98fea52d", - "resource": { - "resourceType": "Observation", - "id": "8e251d63-421d-406e-8cfd-3d6c98fea52d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.81, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8f778cd4-7df9-472d-870e-25920094f962", - "resource": { - "resourceType": "Observation", - "id": "8f778cd4-7df9-472d-870e-25920094f962", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 108.07, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f7c5ce83-f870-42c0-9305-c627ef0a4128", - "resource": { - "resourceType": "Observation", - "id": "f7c5ce83-f870-42c0-9305-c627ef0a4128", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 26.58, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", - "resource": { - "resourceType": "Observation", - "id": "ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 168.67, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b24cd723-57ab-487c-aee1-6d613b11882b", - "resource": { - "resourceType": "Observation", - "id": "b24cd723-57ab-487c-aee1-6d613b11882b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 117.1, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:410421b0-f849-4ef7-ae7f-66e1fb94e5b8", - "resource": { - "resourceType": "Observation", - "id": "410421b0-f849-4ef7-ae7f-66e1fb94e5b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 83.17, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47c45751-fa57-45d4-a317-2ddbad218a1d", - "resource": { - "resourceType": "Observation", - "id": "47c45751-fa57-45d4-a317-2ddbad218a1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 62.07, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aa4b8ede-2213-4a33-a72a-f474629cc496", - "resource": { - "resourceType": "Observation", - "id": "aa4b8ede-2213-4a33-a72a-f474629cc496", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 7.575, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", - "resource": { - "resourceType": "Observation", - "id": "f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.871, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ccc54049-a714-4695-991e-a5a82834371d", - "resource": { - "resourceType": "Observation", - "id": "ccc54049-a714-4695-991e-a5a82834371d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 16.27, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", - "resource": { - "resourceType": "Observation", - "id": "8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 38.741, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", - "resource": { - "resourceType": "Observation", - "id": "fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 92.586, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", - "resource": { - "resourceType": "Observation", - "id": "1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 32.303, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:101e6eca-5997-42fd-b505-46a828396acc", - "resource": { - "resourceType": "Observation", - "id": "101e6eca-5997-42fd-b505-46a828396acc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 34.605, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d96c106b-54e1-45d4-887b-914e335e0150", - "resource": { - "resourceType": "Observation", - "id": "d96c106b-54e1-45d4-887b-914e335e0150", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 43.135, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", - "resource": { - "resourceType": "Observation", - "id": "9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 315, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", - "resource": { - "resourceType": "Observation", - "id": "0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 361.61, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59e54080-4536-4457-bdec-c8a220db1104", - "resource": { - "resourceType": "Observation", - "id": "59e54080-4536-4457-bdec-c8a220db1104", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 11.758, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff271586-f7a8-4b3d-9ab0-3a4a42ad56c2", - "resource": { - "resourceType": "Observation", - "id": "ff271586-f7a8-4b3d-9ab0-3a4a42ad56c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e114920-954b-4c05-972f-eafa39d65d74", - "resource": { - "resourceType": "Observation", - "id": "5e114920-954b-4c05-972f-eafa39d65d74", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.16, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59560246-0d32-426f-87f1-eb44a0ad9555", - "resource": { - "resourceType": "Procedure", - "id": "59560246-0d32-426f-87f1-eb44a0ad9555", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "performedPeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2165251f-c481-4462-8195-f3e298223329", - "resource": { - "resourceType": "MedicationRequest", - "id": "2165251f-c481-4462-8195-f3e298223329", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "authoredOn": "2010-12-05T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:bec69577-e95a-41b4-9398-7e4b2caabe24", - "resource": { - "resourceType": "Claim", - "id": "bec69577-e95a-41b4-9398-7e4b2caabe24", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "created": "2010-12-05T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:2165251f-c481-4462-8195-f3e298223329" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - } - ] - } - ], - "total": { - "value": 17.12, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:12b373cb-db80-4db3-9c55-770cc136b6e8", - "resource": { - "resourceType": "Immunization", - "id": "12b373cb-db80-4db3-9c55-770cc136b6e8", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "occurrenceDateTime": "2010-12-05T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c88c1240-10ed-47ac-9d38-50a26a1af4e5", - "resource": { - "resourceType": "Immunization", - "id": "c88c1240-10ed-47ac-9d38-50a26a1af4e5", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "occurrenceDateTime": "2010-12-05T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1b9f6afa-3e91-46ef-886b-2fbb0e2f7102", - "resource": { - "resourceType": "DiagnosticReport", - "id": "1b9f6afa-3e91-46ef-886b-2fbb0e2f7102", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:7be083f3-8f42-40a8-9c36-1ca04255875f", - "display": "Glucose" - }, - { - "reference": "urn:uuid:3777fdb4-4e26-441b-8760-59bf63aef0f0", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:a04df17d-795d-4971-b74d-605058b14505", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:29bc2948-0bed-40b6-9ac2-22e6129eef77", - "display": "Calcium" - }, - { - "reference": "urn:uuid:81f6ca85-5291-4753-8246-86bdfa0d8779", - "display": "Sodium" - }, - { - "reference": "urn:uuid:8e251d63-421d-406e-8cfd-3d6c98fea52d", - "display": "Potassium" - }, - { - "reference": "urn:uuid:8f778cd4-7df9-472d-870e-25920094f962", - "display": "Chloride" - }, - { - "reference": "urn:uuid:f7c5ce83-f870-42c0-9305-c627ef0a4128", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:71499e71-9aaf-4300-80e1-1d24d530136e", - "resource": { - "resourceType": "DiagnosticReport", - "id": "71499e71-9aaf-4300-80e1-1d24d530136e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:ecfbdcdc-f9f0-4294-9592-3f4f4a4fa5fe", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:b24cd723-57ab-487c-aee1-6d613b11882b", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:410421b0-f849-4ef7-ae7f-66e1fb94e5b8", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:47c45751-fa57-45d4-a317-2ddbad218a1d", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:524e8995-4d9b-4def-b494-8e20df502c9b", - "resource": { - "resourceType": "DiagnosticReport", - "id": "524e8995-4d9b-4def-b494-8e20df502c9b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - }, - "effectiveDateTime": "2010-12-05T22:43:39-08:00", - "issued": "2010-12-05T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:aa4b8ede-2213-4a33-a72a-f474629cc496", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:f7e4dee3-d14c-4980-a95e-0aa3fb194a9a", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:ccc54049-a714-4695-991e-a5a82834371d", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:8aa1d4bd-bd99-4f91-925e-08daf99bc3bb", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:fa4f3627-5ee9-47ce-9b6a-0a85b2e5672d", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:1c1f26cb-445b-4f5e-89d7-bad13b75e2f6", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:101e6eca-5997-42fd-b505-46a828396acc", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:d96c106b-54e1-45d4-887b-914e335e0150", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:9dc5f0f8-4f7f-4d91-aea2-4a4d46e4b025", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:0b909b03-901c-4e8a-9b3a-0d3d7ac6d1b9", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:59e54080-4536-4457-bdec-c8a220db1104", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e", - "resource": { - "resourceType": "Claim", - "id": "dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "created": "2010-12-05T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:12b373cb-db80-4db3-9c55-770cc136b6e8" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c88c1240-10ed-47ac-9d38-50a26a1af4e5" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:59560246-0d32-426f-87f1-eb44a0ad9555" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 4806.21, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:954efde3-c006-412e-9312-5beafd9bb414", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "954efde3-c006-412e-9312-5beafd9bb414", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2010-12-05T23:13:39-08:00", - "end": "2011-12-05T23:13:39-08:00" - }, - "created": "2010-12-05T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:dc9fbdb8-856b-43be-bad0-a6a3eaa5f81e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b0e2b0cd-72e8-4b97-8dd2-6e388c01e524" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2010-12-05T22:43:39-08:00", - "end": "2010-12-05T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 4806.21, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 961.2420000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3844.9680000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 4806.21, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 4806.21, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 4069.8, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab", - "resource": { - "resourceType": "Encounter", - "id": "484f521b-b109-49a8-91c3-f11ccae0e8ab", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:52:39-07:00" - }, - "individual": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "display": "Dr. Dovie983 Mayert710" - } - } - ], - "period": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:52:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9ac27740-54c8-455f-b11b-0b0279c75fbd", - "resource": { - "resourceType": "Procedure", - "id": "9ac27740-54c8-455f-b11b-0b0279c75fbd", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68254000", - "display": "Removal of intrauterine device" - } - ], - "text": "Removal of intrauterine device" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" - }, - "performedPeriod": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:37:39-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5351d455-24d9-46d2-a7c0-2a30405f2181", - "resource": { - "resourceType": "Claim", - "id": "5351d455-24d9-46d2-a7c0-2a30405f2181", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:52:39-07:00" - }, - "created": "2011-06-27T00:52:39-07:00", - "provider": { - "reference": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "display": "MASSACHUSETTS EYE AND EAR INFIRMARY -" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:9ac27740-54c8-455f-b11b-0b0279c75fbd" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - }, - "encounter": [ - { - "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68254000", - "display": "Removal of intrauterine device" - } - ], - "text": "Removal of intrauterine device" - }, - "net": { - "value": 9331.51, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c4c486df-6fa3-44d6-8235-5c4a9d8af510", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c4c486df-6fa3-44d6-8235-5c4a9d8af510", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "performer": [ - { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5351d455-24d9-46d2-a7c0-2a30405f2181" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2011-06-27T00:52:39-07:00", - "end": "2012-06-27T00:52:39-07:00" - }, - "created": "2011-06-27T00:52:39-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5351d455-24d9-46d2-a7c0-2a30405f2181" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - }, - "servicedPeriod": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:52:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:484f521b-b109-49a8-91c3-f11ccae0e8ab" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68254000", - "display": "Removal of intrauterine device" - } - ], - "text": "Removal of intrauterine device" - }, - "servicedPeriod": { - "start": "2011-06-26T23:43:39-07:00", - "end": "2011-06-27T00:52:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 9331.51, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1866.3020000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7465.2080000000005, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 9331.51, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 9331.51, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7465.2080000000005, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0", - "resource": { - "resourceType": "Encounter", - "id": "c6beb7cf-35b9-415c-9dd0-130717faa8a0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1e88c727-6049-4a3f-89c1-e29d93447e25", - "resource": { - "resourceType": "Observation", - "id": "1e88c727-6049-4a3f-89c1-e29d93447e25", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 55.559, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5ced0f5-ec47-403d-b05c-76730f066dbf", - "resource": { - "resourceType": "Observation", - "id": "f5ced0f5-ec47-403d-b05c-76730f066dbf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f553a64-f551-42fc-9f41-daee856da6f5", - "resource": { - "resourceType": "Observation", - "id": "5f553a64-f551-42fc-9f41-daee856da6f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e9399377-66d1-4799-860d-f5a31f3ee5db", - "resource": { - "resourceType": "Observation", - "id": "e9399377-66d1-4799-860d-f5a31f3ee5db", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 73.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:79862766-83c1-43a4-98ac-03ee8a39aa39", - "resource": { - "resourceType": "Observation", - "id": "79862766-83c1-43a4-98ac-03ee8a39aa39", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 28.27, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4db6a529-675e-4a63-b965-53c9e8abfa8a", - "resource": { - "resourceType": "Observation", - "id": "4db6a529-675e-4a63-b965-53c9e8abfa8a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 117, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d340645d-91f9-43ad-961e-90a4d5413f0a", - "resource": { - "resourceType": "Observation", - "id": "d340645d-91f9-43ad-961e-90a4d5413f0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 92, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67a031c1-ffa4-4acc-8585-ad05de1f53df", - "resource": { - "resourceType": "Observation", - "id": "67a031c1-ffa4-4acc-8585-ad05de1f53df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:44547337-d65d-4939-b6a1-e50697a91474", - "resource": { - "resourceType": "Observation", - "id": "44547337-d65d-4939-b6a1-e50697a91474", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 84.64, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:621e86d2-a98c-40bf-adbe-8ff57d0004e8", - "resource": { - "resourceType": "Observation", - "id": "621e86d2-a98c-40bf-adbe-8ff57d0004e8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 17.21, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6caaac39-fa12-4e31-a101-c035df4d7943", - "resource": { - "resourceType": "Observation", - "id": "6caaac39-fa12-4e31-a101-c035df4d7943", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.77, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dfcbfbd2-13b5-41dc-82db-0b102079d64d", - "resource": { - "resourceType": "Observation", - "id": "dfcbfbd2-13b5-41dc-82db-0b102079d64d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.95, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f5446c7-f8a3-41db-a24c-1b76f535030a", - "resource": { - "resourceType": "Observation", - "id": "5f5446c7-f8a3-41db-a24c-1b76f535030a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 140.55, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d56c3b65-6525-4e02-9f95-19034fb17719", - "resource": { - "resourceType": "Observation", - "id": "d56c3b65-6525-4e02-9f95-19034fb17719", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 5, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:52ea8a78-ee19-4767-bded-ba61a20b704a", - "resource": { - "resourceType": "Observation", - "id": "52ea8a78-ee19-4767-bded-ba61a20b704a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 106.34, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d959c9b-95e5-4cc6-b50a-42faaa6e6308", - "resource": { - "resourceType": "Observation", - "id": "2d959c9b-95e5-4cc6-b50a-42faaa6e6308", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 21.61, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d44dbd7-d7b3-42d6-bdc8-419752261db5", - "resource": { - "resourceType": "Observation", - "id": "7d44dbd7-d7b3-42d6-bdc8-419752261db5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:80bb5b7a-6010-4f7a-9813-5eae64a461fa", - "resource": { - "resourceType": "Observation", - "id": "80bb5b7a-6010-4f7a-9813-5eae64a461fa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.36, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8b46f00-7b17-46a2-9d21-6d91f53dc8df", - "resource": { - "resourceType": "Procedure", - "id": "e8b46f00-7b17-46a2-9d21-6d91f53dc8df", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "performedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c3a06cc9-a555-4b36-b624-44aeb90b1ba3", - "resource": { - "resourceType": "MedicationRequest", - "id": "c3a06cc9-a555-4b36-b624-44aeb90b1ba3", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "authoredOn": "2011-12-11T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:c6b5665b-0ba0-458e-b8d3-095909e90b1d", - "resource": { - "resourceType": "Claim", - "id": "c6b5665b-0ba0-458e-b8d3-095909e90b1d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "created": "2011-12-11T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:c3a06cc9-a555-4b36-b624-44aeb90b1ba3" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - } - ] - } - ], - "total": { - "value": 23.82, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7baf31f8-b469-4a81-a8ff-a094c49df212", - "resource": { - "resourceType": "Immunization", - "id": "7baf31f8-b469-4a81-a8ff-a094c49df212", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "occurrenceDateTime": "2011-12-11T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8fa0ff7e-f7c2-4719-b9b6-620e15507668", - "resource": { - "resourceType": "Immunization", - "id": "8fa0ff7e-f7c2-4719-b9b6-620e15507668", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "occurrenceDateTime": "2011-12-11T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:6bf98825-4c89-42c7-bc08-671bce82c6b7", - "resource": { - "resourceType": "Immunization", - "id": "6bf98825-4c89-42c7-bc08-671bce82c6b7", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "occurrenceDateTime": "2011-12-11T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:855bbd86-3835-408d-92de-0ffa9986a8a2", - "resource": { - "resourceType": "DiagnosticReport", - "id": "855bbd86-3835-408d-92de-0ffa9986a8a2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - }, - "effectiveDateTime": "2011-12-11T22:43:39-08:00", - "issued": "2011-12-11T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:44547337-d65d-4939-b6a1-e50697a91474", - "display": "Glucose" - }, - { - "reference": "urn:uuid:621e86d2-a98c-40bf-adbe-8ff57d0004e8", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:6caaac39-fa12-4e31-a101-c035df4d7943", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:dfcbfbd2-13b5-41dc-82db-0b102079d64d", - "display": "Calcium" - }, - { - "reference": "urn:uuid:5f5446c7-f8a3-41db-a24c-1b76f535030a", - "display": "Sodium" - }, - { - "reference": "urn:uuid:d56c3b65-6525-4e02-9f95-19034fb17719", - "display": "Potassium" - }, - { - "reference": "urn:uuid:52ea8a78-ee19-4767-bded-ba61a20b704a", - "display": "Chloride" - }, - { - "reference": "urn:uuid:2d959c9b-95e5-4cc6-b50a-42faaa6e6308", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:aa77a4ce-50ba-4049-8866-f430d9c05879", - "resource": { - "resourceType": "Claim", - "id": "aa77a4ce-50ba-4049-8866-f430d9c05879", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "created": "2011-12-11T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7baf31f8-b469-4a81-a8ff-a094c49df212" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8fa0ff7e-f7c2-4719-b9b6-620e15507668" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:6bf98825-4c89-42c7-bc08-671bce82c6b7" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e8b46f00-7b17-46a2-9d21-6d91f53dc8df" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 5, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 7845.43, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:aa7595b7-f53d-4069-9382-5965eaafea81", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "aa7595b7-f53d-4069-9382-5965eaafea81", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "aa77a4ce-50ba-4049-8866-f430d9c05879" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2011-12-11T23:13:39-08:00", - "end": "2012-12-11T23:13:39-08:00" - }, - "created": "2011-12-11T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:aa77a4ce-50ba-4049-8866-f430d9c05879" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c6beb7cf-35b9-415c-9dd0-130717faa8a0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "servicedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2011-12-11T22:43:39-08:00", - "end": "2011-12-11T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 7845.43, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1569.0860000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 6276.344000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 7845.43, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 7845.43, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 6613.592000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955", - "resource": { - "resourceType": "Encounter", - "id": "fd5aa686-a8ac-4a82-bf0e-50974feec955", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-01-22T22:43:39-08:00", - "end": "2012-01-22T22:58:39-08:00" - }, - "individual": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "display": "Dr. Max124 Howell947" - } - } - ], - "period": { - "start": "2012-01-22T22:43:39-08:00", - "end": "2012-01-22T22:58:39-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23", - "resource": { - "resourceType": "Condition", - "id": "aa3202d2-c88e-4f71-a68c-d9ca53df9a23", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" - }, - "onsetDateTime": "2012-01-22T22:43:39-08:00", - "abatementDateTime": "2012-01-29T22:43:39-08:00", - "recordedDate": "2012-01-22T22:43:39-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:8aabc23e-3685-4406-92f2-8f94644170d6", - "resource": { - "resourceType": "Claim", - "id": "8aabc23e-3685-4406-92f2-8f94644170d6", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2012-01-22T22:43:39-08:00", - "end": "2012-01-22T22:58:39-08:00" - }, - "created": "2012-01-22T22:58:39-08:00", - "provider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b0ac1954-fcff-4e04-8c3c-ecc9d9cc3b79", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b0ac1954-fcff-4e04-8c3c-ecc9d9cc3b79", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "performer": [ - { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8aabc23e-3685-4406-92f2-8f94644170d6" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2012-01-22T22:58:39-08:00", - "end": "2013-01-22T22:58:39-08:00" - }, - "created": "2012-01-22T22:58:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8aabc23e-3685-4406-92f2-8f94644170d6" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:aa3202d2-c88e-4f71-a68c-d9ca53df9a23" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2012-01-22T22:43:39-08:00", - "end": "2012-01-22T22:58:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:fd5aa686-a8ac-4a82-bf0e-50974feec955" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "servicedPeriod": { - "start": "2012-01-22T22:43:39-08:00", - "end": "2012-01-22T22:58:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b", - "resource": { - "resourceType": "Encounter", - "id": "bd35fdbc-a30b-4a4d-aa39-41f7395d518b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2bb6c753-4248-464e-bde8-8029b4ffb551", - "resource": { - "resourceType": "Observation", - "id": "2bb6c753-4248-464e-bde8-8029b4ffb551", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 68.222, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8590cf5-2d61-4c67-b240-8073ced3df9d", - "resource": { - "resourceType": "Observation", - "id": "e8590cf5-2d61-4c67-b240-8073ced3df9d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2844371d-8829-4f08-bcdf-77b5807fcf02", - "resource": { - "resourceType": "Observation", - "id": "2844371d-8829-4f08-bcdf-77b5807fcf02", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33ab88c0-4e17-4689-b5f4-f9ef46c6520f", - "resource": { - "resourceType": "Observation", - "id": "33ab88c0-4e17-4689-b5f4-f9ef46c6520f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 75.2, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:659107a5-2495-4f01-906e-ba5b6d6c772e", - "resource": { - "resourceType": "Observation", - "id": "659107a5-2495-4f01-906e-ba5b6d6c772e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 29.01, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3cc5422-9dc6-422d-a89f-8d1888c06a59", - "resource": { - "resourceType": "Observation", - "id": "f3cc5422-9dc6-422d-a89f-8d1888c06a59", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 120, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6055ab5c-2274-4517-baaa-b2a643471122", - "resource": { - "resourceType": "Observation", - "id": "6055ab5c-2274-4517-baaa-b2a643471122", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 61, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:feb63539-8dfe-4d46-9075-f1bfb1ff28b3", - "resource": { - "resourceType": "Observation", - "id": "feb63539-8dfe-4d46-9075-f1bfb1ff28b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:046551b5-8be1-44ec-aeae-a01550e2a840", - "resource": { - "resourceType": "Observation", - "id": "046551b5-8be1-44ec-aeae-a01550e2a840", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 90.98, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:232470a3-b85c-4692-93d3-8ee2208dd6c7", - "resource": { - "resourceType": "Observation", - "id": "232470a3-b85c-4692-93d3-8ee2208dd6c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 18.04, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d3d23ee-4387-4281-80b5-ace382eda27f", - "resource": { - "resourceType": "Observation", - "id": "1d3d23ee-4387-4281-80b5-ace382eda27f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 1.22, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59364e97-3c77-4951-a226-5993ee64a447", - "resource": { - "resourceType": "Observation", - "id": "59364e97-3c77-4951-a226-5993ee64a447", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 8.74, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33129f88-4c75-4b90-a9a5-115d2c460712", - "resource": { - "resourceType": "Observation", - "id": "33129f88-4c75-4b90-a9a5-115d2c460712", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 140.33, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1bcc7db0-f025-4205-a6d9-625457a8a7c7", - "resource": { - "resourceType": "Observation", - "id": "1bcc7db0-f025-4205-a6d9-625457a8a7c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.11, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:04c373c5-2765-4e1c-99b4-690b02661135", - "resource": { - "resourceType": "Observation", - "id": "04c373c5-2765-4e1c-99b4-690b02661135", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 110.6, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfb00c59-e640-4531-9dff-b17f46fee964", - "resource": { - "resourceType": "Observation", - "id": "cfb00c59-e640-4531-9dff-b17f46fee964", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 22.67, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8ddfdc1c-ee9c-4887-b377-2f9049c273c8", - "resource": { - "resourceType": "Observation", - "id": "8ddfdc1c-ee9c-4887-b377-2f9049c273c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:81165715-4e9d-44e8-bad0-db827c8c09f6", - "resource": { - "resourceType": "Observation", - "id": "81165715-4e9d-44e8-bad0-db827c8c09f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.39, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d4a5c25-9381-4446-8338-54a129f342b4", - "resource": { - "resourceType": "Procedure", - "id": "0d4a5c25-9381-4446-8338-54a129f342b4", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "performedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ed7a4b76-8540-4854-bb88-5189a9946a9d", - "resource": { - "resourceType": "Procedure", - "id": "ed7a4b76-8540-4854-bb88-5189a9946a9d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "performedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T22:58:39-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:49b07d1a-14a4-4f59-a8f2-a2164030714c", - "resource": { - "resourceType": "MedicationRequest", - "id": "49b07d1a-14a4-4f59-a8f2-a2164030714c", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "authoredOn": "2012-12-16T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:552399db-be53-4538-a0cc-bae798c5b644", - "resource": { - "resourceType": "Claim", - "id": "552399db-be53-4538-a0cc-bae798c5b644", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "created": "2012-12-16T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:49b07d1a-14a4-4f59-a8f2-a2164030714c" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - } - ] - } - ], - "total": { - "value": 26.77, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:390d154c-e694-439f-982f-2f327c2bc201", - "resource": { - "resourceType": "Immunization", - "id": "390d154c-e694-439f-982f-2f327c2bc201", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "occurrenceDateTime": "2012-12-16T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:75939c84-7884-43d2-ac63-9b23551728d9", - "resource": { - "resourceType": "DiagnosticReport", - "id": "75939c84-7884-43d2-ac63-9b23551728d9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - }, - "effectiveDateTime": "2012-12-16T22:43:39-08:00", - "issued": "2012-12-16T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:046551b5-8be1-44ec-aeae-a01550e2a840", - "display": "Glucose" - }, - { - "reference": "urn:uuid:232470a3-b85c-4692-93d3-8ee2208dd6c7", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:1d3d23ee-4387-4281-80b5-ace382eda27f", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:59364e97-3c77-4951-a226-5993ee64a447", - "display": "Calcium" - }, - { - "reference": "urn:uuid:33129f88-4c75-4b90-a9a5-115d2c460712", - "display": "Sodium" - }, - { - "reference": "urn:uuid:1bcc7db0-f025-4205-a6d9-625457a8a7c7", - "display": "Potassium" - }, - { - "reference": "urn:uuid:04c373c5-2765-4e1c-99b4-690b02661135", - "display": "Chloride" - }, - { - "reference": "urn:uuid:cfb00c59-e640-4531-9dff-b17f46fee964", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:e9d6ffeb-ce12-4703-b888-380d272ffa78", - "resource": { - "resourceType": "Claim", - "id": "e9d6ffeb-ce12-4703-b888-380d272ffa78", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "created": "2012-12-16T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:390d154c-e694-439f-982f-2f327c2bc201" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:0d4a5c25-9381-4446-8338-54a129f342b4" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:ed7a4b76-8540-4854-bb88-5189a9946a9d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 7449.97, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 882.97, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8e38837b-8ebc-4ae2-b74a-f55f1bfa5671", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8e38837b-8ebc-4ae2-b74a-f55f1bfa5671", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e9d6ffeb-ce12-4703-b888-380d272ffa78" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2012-12-16T23:28:39-08:00", - "end": "2013-12-16T23:28:39-08:00" - }, - "created": "2012-12-16T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e9d6ffeb-ce12-4703-b888-380d272ffa78" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:bd35fdbc-a30b-4a4d-aa39-41f7395d518b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 7449.97, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1489.9940000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 5959.976000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 7449.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 7449.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2012-12-16T22:43:39-08:00", - "end": "2012-12-16T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 882.97, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 176.59400000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 706.3760000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 882.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 882.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 6778.768000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9", - "resource": { - "resourceType": "Encounter", - "id": "f52f6536-54a4-4df7-8148-fef06a43eab9", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3e858f82-b1aa-4fa0-941c-dcabeed081f1", - "resource": { - "resourceType": "Observation", - "id": "3e858f82-b1aa-4fa0-941c-dcabeed081f1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 60.593, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:736d1780-ebb7-4ee3-bd76-6e52f759f467", - "resource": { - "resourceType": "Observation", - "id": "736d1780-ebb7-4ee3-bd76-6e52f759f467", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:acbb6001-cb87-48d7-936d-b789e4c7a20f", - "resource": { - "resourceType": "Observation", - "id": "acbb6001-cb87-48d7-936d-b789e4c7a20f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf769eaa-0ca4-4706-abd5-52390613f5fc", - "resource": { - "resourceType": "Observation", - "id": "cf769eaa-0ca4-4706-abd5-52390613f5fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 77.1, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7739c6fb-6da8-464b-93af-f823c7d608f6", - "resource": { - "resourceType": "Observation", - "id": "7739c6fb-6da8-464b-93af-f823c7d608f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 29.76, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4219406-0e09-48b7-a7ab-912034ca8c4b", - "resource": { - "resourceType": "Observation", - "id": "d4219406-0e09-48b7-a7ab-912034ca8c4b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 135, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:52f95900-4e4b-4960-ae4b-4362373187da", - "resource": { - "resourceType": "Observation", - "id": "52f95900-4e4b-4960-ae4b-4362373187da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11796cbf-9dcd-427b-92f0-5f31d124d8a6", - "resource": { - "resourceType": "Observation", - "id": "11796cbf-9dcd-427b-92f0-5f31d124d8a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b08d9700-a684-48df-b6ea-844988b65172", - "resource": { - "resourceType": "Observation", - "id": "b08d9700-a684-48df-b6ea-844988b65172", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 85.01, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6aa2b320-bf22-4b4a-802f-90566174d079", - "resource": { - "resourceType": "Observation", - "id": "6aa2b320-bf22-4b4a-802f-90566174d079", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.45, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cbfc87c2-2d5e-4a0b-93dc-674d697ca838", - "resource": { - "resourceType": "Observation", - "id": "cbfc87c2-2d5e-4a0b-93dc-674d697ca838", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.84, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b23e8e1-0a91-4e70-a06f-7c145b564441", - "resource": { - "resourceType": "Observation", - "id": "8b23e8e1-0a91-4e70-a06f-7c145b564441", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 10.17, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3fc922ac-c871-4716-8378-ee6aebacc37d", - "resource": { - "resourceType": "Observation", - "id": "3fc922ac-c871-4716-8378-ee6aebacc37d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 139.93, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51c9eb84-cd7b-46d2-bf32-bf3b160ade97", - "resource": { - "resourceType": "Observation", - "id": "51c9eb84-cd7b-46d2-bf32-bf3b160ade97", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.31, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:314ab8a1-1b08-462d-a6e9-89972c40ecee", - "resource": { - "resourceType": "Observation", - "id": "314ab8a1-1b08-462d-a6e9-89972c40ecee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 108.41, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ad33c6dd-bd07-431d-9325-f3e2b8857951", - "resource": { - "resourceType": "Observation", - "id": "ad33c6dd-bd07-431d-9325-f3e2b8857951", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 20.44, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4d11c1f-62cb-49bf-9f37-6d20b08daced", - "resource": { - "resourceType": "Observation", - "id": "d4d11c1f-62cb-49bf-9f37-6d20b08daced", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 172.12, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddbbb510-58f8-4659-8248-8307e441e0cc", - "resource": { - "resourceType": "Observation", - "id": "ddbbb510-58f8-4659-8248-8307e441e0cc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 132.2, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ce68f389-c419-476e-9afc-6af644fa2950", - "resource": { - "resourceType": "Observation", - "id": "ce68f389-c419-476e-9afc-6af644fa2950", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 75, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:60f10017-43f7-47d5-a1d9-e77632adf3c0", - "resource": { - "resourceType": "Observation", - "id": "60f10017-43f7-47d5-a1d9-e77632adf3c0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 70.68, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:519a967f-38bd-4fe9-9790-4d2f05492a3e", - "resource": { - "resourceType": "Observation", - "id": "519a967f-38bd-4fe9-9790-4d2f05492a3e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d07d733-489d-40c6-a58f-304bcdc9290b", - "resource": { - "resourceType": "Observation", - "id": "2d07d733-489d-40c6-a58f-304bcdc9290b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.01, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:37d4b309-7901-4acf-9797-968f6bfcab14", - "resource": { - "resourceType": "Procedure", - "id": "37d4b309-7901-4acf-9797-968f6bfcab14", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "performedPeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e33496eb-6003-4555-8933-d39c10aa5bae", - "resource": { - "resourceType": "MedicationRequest", - "id": "e33496eb-6003-4555-8933-d39c10aa5bae", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "authoredOn": "2013-12-22T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:f9fc5cfd-6e2f-4332-a820-79b9199e6113", - "resource": { - "resourceType": "Claim", - "id": "f9fc5cfd-6e2f-4332-a820-79b9199e6113", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "created": "2013-12-22T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:e33496eb-6003-4555-8933-d39c10aa5bae" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - } - ] - } - ], - "total": { - "value": 24.06, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2", - "resource": { - "resourceType": "Immunization", - "id": "f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "occurrenceDateTime": "2013-12-22T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:dbcdc951-84cb-4870-a500-056c8a78aa6d", - "resource": { - "resourceType": "DiagnosticReport", - "id": "dbcdc951-84cb-4870-a500-056c8a78aa6d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:b08d9700-a684-48df-b6ea-844988b65172", - "display": "Glucose" - }, - { - "reference": "urn:uuid:6aa2b320-bf22-4b4a-802f-90566174d079", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:cbfc87c2-2d5e-4a0b-93dc-674d697ca838", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:8b23e8e1-0a91-4e70-a06f-7c145b564441", - "display": "Calcium" - }, - { - "reference": "urn:uuid:3fc922ac-c871-4716-8378-ee6aebacc37d", - "display": "Sodium" - }, - { - "reference": "urn:uuid:51c9eb84-cd7b-46d2-bf32-bf3b160ade97", - "display": "Potassium" - }, - { - "reference": "urn:uuid:314ab8a1-1b08-462d-a6e9-89972c40ecee", - "display": "Chloride" - }, - { - "reference": "urn:uuid:ad33c6dd-bd07-431d-9325-f3e2b8857951", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:4f2de63d-a4f1-4db1-a088-dc8f11fe2bc1", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4f2de63d-a4f1-4db1-a088-dc8f11fe2bc1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - }, - "effectiveDateTime": "2013-12-22T22:43:39-08:00", - "issued": "2013-12-22T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:d4d11c1f-62cb-49bf-9f37-6d20b08daced", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:ddbbb510-58f8-4659-8248-8307e441e0cc", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:ce68f389-c419-476e-9afc-6af644fa2950", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:60f10017-43f7-47d5-a1d9-e77632adf3c0", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d", - "resource": { - "resourceType": "Claim", - "id": "55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "created": "2013-12-22T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f69eeb85-00cb-4560-8c0f-ed3a37fbc6a2" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:37d4b309-7901-4acf-9797-968f6bfcab14" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 8942.99, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e8012c5a-73f8-482f-808b-92a098645d2a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e8012c5a-73f8-482f-808b-92a098645d2a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2013-12-22T23:13:39-08:00", - "end": "2014-12-22T23:13:39-08:00" - }, - "created": "2013-12-22T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:55e40282-8e20-4b3b-ab7e-3c78d6cdaf0d" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f52f6536-54a4-4df7-8148-fef06a43eab9" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2013-12-22T22:43:39-08:00", - "end": "2013-12-22T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 8942.99, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1788.598, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7154.392, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 8942.99, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 8942.99, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7266.808, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce", - "resource": { - "resourceType": "Encounter", - "id": "4d26768d-36e4-4eb5-8c91-73bcbdb251ce", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2d73a3c0-c78b-4456-a71d-443c2d29f9c3", - "resource": { - "resourceType": "Observation", - "id": "2d73a3c0-c78b-4456-a71d-443c2d29f9c3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 63.383, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a37713f0-c64c-476b-a543-0d4d9762a5b8", - "resource": { - "resourceType": "Observation", - "id": "a37713f0-c64c-476b-a543-0d4d9762a5b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b5eae6e-69bb-442f-908a-ffc6425516e5", - "resource": { - "resourceType": "Observation", - "id": "1b5eae6e-69bb-442f-908a-ffc6425516e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d24ff37-f10a-4906-a41a-1bbe779567e0", - "resource": { - "resourceType": "Observation", - "id": "0d24ff37-f10a-4906-a41a-1bbe779567e0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82f79397-68d4-4ee3-a60b-db191e4f1d35", - "resource": { - "resourceType": "Observation", - "id": "82f79397-68d4-4ee3-a60b-db191e4f1d35", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7cf4f1ee-59f4-446a-89bd-a7ad36fd4645", - "resource": { - "resourceType": "Observation", - "id": "7cf4f1ee-59f4-446a-89bd-a7ad36fd4645", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:928bbf24-634a-48c3-af65-dde6309b2ed9", - "resource": { - "resourceType": "Observation", - "id": "928bbf24-634a-48c3-af65-dde6309b2ed9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92049736-d9d4-4b69-bd9a-7e2f12239900", - "resource": { - "resourceType": "Observation", - "id": "92049736-d9d4-4b69-bd9a-7e2f12239900", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:89badd04-b9a1-4dfd-a31f-8f736a93402f", - "resource": { - "resourceType": "Observation", - "id": "89badd04-b9a1-4dfd-a31f-8f736a93402f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 85.07, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a875b76d-0043-4352-91f5-19ca9a8c3fcc", - "resource": { - "resourceType": "Observation", - "id": "a875b76d-0043-4352-91f5-19ca9a8c3fcc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 12.47, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:582b5b9b-e61d-41d6-958b-8d1af0e32a31", - "resource": { - "resourceType": "Observation", - "id": "582b5b9b-e61d-41d6-958b-8d1af0e32a31", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.8, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33572d1a-c496-4821-926f-037d901744f1", - "resource": { - "resourceType": "Observation", - "id": "33572d1a-c496-4821-926f-037d901744f1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.23, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", - "resource": { - "resourceType": "Observation", - "id": "68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 141.6, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:18a1e7e8-97ea-413a-afcd-71a9738dfc93", - "resource": { - "resourceType": "Observation", - "id": "18a1e7e8-97ea-413a-afcd-71a9738dfc93", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.39, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", - "resource": { - "resourceType": "Observation", - "id": "e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 102.89, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7911c2f1-afbe-4315-ba4b-b9b3e852027c", - "resource": { - "resourceType": "Observation", - "id": "7911c2f1-afbe-4315-ba4b-b9b3e852027c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 20.44, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d548fcc-9075-4a54-8ba8-d8c1f44fe56e", - "resource": { - "resourceType": "Observation", - "id": "3d548fcc-9075-4a54-8ba8-d8c1f44fe56e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9e294388-b1a3-4e2a-9f84-35355b7682db", - "resource": { - "resourceType": "Observation", - "id": "9e294388-b1a3-4e2a-9f84-35355b7682db", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.19, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c5c47bc9-3482-42eb-9aaf-1aeb90672c88", - "resource": { - "resourceType": "Procedure", - "id": "c5c47bc9-3482-42eb-9aaf-1aeb90672c88", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "performedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:35533f62-fa1a-4a62-bb02-a7179da222f0", - "resource": { - "resourceType": "Procedure", - "id": "35533f62-fa1a-4a62-bb02-a7179da222f0", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "performedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T22:58:39-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2674fcf5-9e55-4567-ba6a-54f9aeb4b88c", - "resource": { - "resourceType": "MedicationRequest", - "id": "2674fcf5-9e55-4567-ba6a-54f9aeb4b88c", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "authoredOn": "2014-12-28T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5244ddc1-58fe-4891-8e37-efca4e81e649", - "resource": { - "resourceType": "Claim", - "id": "5244ddc1-58fe-4891-8e37-efca4e81e649", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "created": "2014-12-28T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:2674fcf5-9e55-4567-ba6a-54f9aeb4b88c" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - } - ] - } - ], - "total": { - "value": 47.4, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:62640e6f-d0ed-4ddc-b104-9ada341ec832", - "resource": { - "resourceType": "Immunization", - "id": "62640e6f-d0ed-4ddc-b104-9ada341ec832", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "occurrenceDateTime": "2014-12-28T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7abeb6dc-e804-4498-988b-7e0b9035fdfc", - "resource": { - "resourceType": "DiagnosticReport", - "id": "7abeb6dc-e804-4498-988b-7e0b9035fdfc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - }, - "effectiveDateTime": "2014-12-28T22:43:39-08:00", - "issued": "2014-12-28T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:89badd04-b9a1-4dfd-a31f-8f736a93402f", - "display": "Glucose" - }, - { - "reference": "urn:uuid:a875b76d-0043-4352-91f5-19ca9a8c3fcc", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:582b5b9b-e61d-41d6-958b-8d1af0e32a31", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:33572d1a-c496-4821-926f-037d901744f1", - "display": "Calcium" - }, - { - "reference": "urn:uuid:68a90deb-a5f8-4e72-8d97-054e3cfc5c5d", - "display": "Sodium" - }, - { - "reference": "urn:uuid:18a1e7e8-97ea-413a-afcd-71a9738dfc93", - "display": "Potassium" - }, - { - "reference": "urn:uuid:e7bb6e9d-33fa-4dd8-b3e1-b332db4dea0a", - "display": "Chloride" - }, - { - "reference": "urn:uuid:7911c2f1-afbe-4315-ba4b-b9b3e852027c", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:89b59121-6403-4820-bd2d-86bae1fd8afe", - "resource": { - "resourceType": "Claim", - "id": "89b59121-6403-4820-bd2d-86bae1fd8afe", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "created": "2014-12-28T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:62640e6f-d0ed-4ddc-b104-9ada341ec832" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:c5c47bc9-3482-42eb-9aaf-1aeb90672c88" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:35533f62-fa1a-4a62-bb02-a7179da222f0" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 6270.22, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 360.08, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8f28d0f0-1855-4231-befc-33ac6bfde047", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8f28d0f0-1855-4231-befc-33ac6bfde047", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "89b59121-6403-4820-bd2d-86bae1fd8afe" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2014-12-28T23:28:39-08:00", - "end": "2015-12-28T23:28:39-08:00" - }, - "created": "2014-12-28T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:89b59121-6403-4820-bd2d-86bae1fd8afe" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4d26768d-36e4-4eb5-8c91-73bcbdb251ce" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 6270.22, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1254.044, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 5016.176, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 6270.22, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 6270.22, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2014-12-28T22:43:39-08:00", - "end": "2014-12-28T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 360.08, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 72.016, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 288.064, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 360.08, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 360.08, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 5416.656000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019", - "resource": { - "resourceType": "Encounter", - "id": "fabd54e8-f98b-43ab-8abf-c966701b6019", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:41:39-07:00" - }, - "individual": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "display": "Dr. Max124 Howell947" - } - } - ], - "period": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:41:39-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9067491c-015c-430b-a246-4c37fe2f8a09", - "resource": { - "resourceType": "Procedure", - "id": "9067491c-015c-430b-a246-4c37fe2f8a09", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" - }, - "performedPeriod": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:26:39-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3b2d05f9-7128-4b54-b24c-4220fef99a10", - "resource": { - "resourceType": "Claim", - "id": "3b2d05f9-7128-4b54-b24c-4220fef99a10", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:41:39-07:00" - }, - "created": "2015-10-03T00:41:39-07:00", - "provider": { - "reference": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "display": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:9067491c-015c-430b-a246-4c37fe2f8a09" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 11770.61, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:01ba71a3-8baa-4589-ab87-23323a43584a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "01ba71a3-8baa-4589-ab87-23323a43584a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "performer": [ - { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3b2d05f9-7128-4b54-b24c-4220fef99a10" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2015-10-03T00:41:39-07:00", - "end": "2016-10-03T00:41:39-07:00" - }, - "created": "2015-10-03T00:41:39-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3b2d05f9-7128-4b54-b24c-4220fef99a10" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:41:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:fabd54e8-f98b-43ab-8abf-c966701b6019" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2015-10-02T23:43:39-07:00", - "end": "2015-10-03T00:41:39-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 11770.61, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2354.1220000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9416.488000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11770.61, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11770.61, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 9416.488000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e", - "resource": { - "resourceType": "Encounter", - "id": "8e3c7581-a34a-439c-8a9c-6bfea816ca4e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bfb3ffa2-d465-474d-b342-b15bd99c7d5a", - "resource": { - "resourceType": "Observation", - "id": "bfb3ffa2-d465-474d-b342-b15bd99c7d5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 71.731, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:60815011-e762-44a9-97bb-5522fb436590", - "resource": { - "resourceType": "Observation", - "id": "60815011-e762-44a9-97bb-5522fb436590", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:233ae8c8-7248-4730-9e87-125a986a581e", - "resource": { - "resourceType": "Observation", - "id": "233ae8c8-7248-4730-9e87-125a986a581e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:055fb244-41ee-4ba1-9257-136dadbb2ee4", - "resource": { - "resourceType": "Observation", - "id": "055fb244-41ee-4ba1-9257-136dadbb2ee4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6f9ee673-9a76-44bd-9611-abad79fa1744", - "resource": { - "resourceType": "Observation", - "id": "6f9ee673-9a76-44bd-9611-abad79fa1744", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:007d321f-df2f-4540-b8fc-fbf0e4fc812a", - "resource": { - "resourceType": "Observation", - "id": "007d321f-df2f-4540-b8fc-fbf0e4fc812a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 131, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e1528803-f67a-4609-8625-c1003d848f99", - "resource": { - "resourceType": "Observation", - "id": "e1528803-f67a-4609-8625-c1003d848f99", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 90, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac789242-35e2-47e0-b590-4ccc991397f9", - "resource": { - "resourceType": "Observation", - "id": "ac789242-35e2-47e0-b590-4ccc991397f9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:00915ad1-e8da-44b4-b216-93153231b84a", - "resource": { - "resourceType": "Observation", - "id": "00915ad1-e8da-44b4-b216-93153231b84a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 72.36, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ea0ab9c0-3864-4af7-9488-5750a22fe826", - "resource": { - "resourceType": "Observation", - "id": "ea0ab9c0-3864-4af7-9488-5750a22fe826", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 17.89, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5687f0d4-bbd0-402c-84af-f91d4cce566f", - "resource": { - "resourceType": "Observation", - "id": "5687f0d4-bbd0-402c-84af-f91d4cce566f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.86, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5f2abb5-10ad-4914-bb83-5f38c8e827f3", - "resource": { - "resourceType": "Observation", - "id": "f5f2abb5-10ad-4914-bb83-5f38c8e827f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.34, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f762dfda-ffbf-4d97-8ae9-564213b6acb6", - "resource": { - "resourceType": "Observation", - "id": "f762dfda-ffbf-4d97-8ae9-564213b6acb6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 138.48, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", - "resource": { - "resourceType": "Observation", - "id": "ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.08, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9b9877da-aad6-46b9-99e6-cc80aa86b1c8", - "resource": { - "resourceType": "Observation", - "id": "9b9877da-aad6-46b9-99e6-cc80aa86b1c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 105.9, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:493372ae-260b-412a-b3b7-461278f3cb77", - "resource": { - "resourceType": "Observation", - "id": "493372ae-260b-412a-b3b7-461278f3cb77", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 21.1, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a61b2a3-bd85-421c-96c2-c6eb382ea08e", - "resource": { - "resourceType": "Observation", - "id": "5a61b2a3-bd85-421c-96c2-c6eb382ea08e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 7.7907, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:874a2c15-1c59-44de-8821-96d9f1e0a8da", - "resource": { - "resourceType": "Observation", - "id": "874a2c15-1c59-44de-8821-96d9f1e0a8da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 3.9418, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df6499d7-8b55-43c4-b7df-efbdbcd67a09", - "resource": { - "resourceType": "Observation", - "id": "df6499d7-8b55-43c4-b7df-efbdbcd67a09", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 17.262, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d3a76fa9-be44-40d0-8ddc-a4b96a18a664", - "resource": { - "resourceType": "Observation", - "id": "d3a76fa9-be44-40d0-8ddc-a4b96a18a664", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 35.906, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5635958-2b38-4d0f-9de1-60956c7824dc", - "resource": { - "resourceType": "Observation", - "id": "f5635958-2b38-4d0f-9de1-60956c7824dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 93.807, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9553d8a-a97c-449d-a201-7a8bde168a89", - "resource": { - "resourceType": "Observation", - "id": "c9553d8a-a97c-449d-a201-7a8bde168a89", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.129, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d6768ab4-a243-40fb-b6aa-923ea3154778", - "resource": { - "resourceType": "Observation", - "id": "d6768ab4-a243-40fb-b6aa-923ea3154778", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 34.413, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3155135a-faa8-44be-abd2-56e1490dded8", - "resource": { - "resourceType": "Observation", - "id": "3155135a-faa8-44be-abd2-56e1490dded8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 45.889, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebd1df5e-a263-42be-bc0f-7e5e80807537", - "resource": { - "resourceType": "Observation", - "id": "ebd1df5e-a263-42be-bc0f-7e5e80807537", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 394.04, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", - "resource": { - "resourceType": "Observation", - "id": "fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 415.07, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bc6a414e-d54e-4553-a22b-726d4786977d", - "resource": { - "resourceType": "Observation", - "id": "bc6a414e-d54e-4553-a22b-726d4786977d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 10.033, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1073d9e6-4230-421f-95b2-d94e5cb6ba1b", - "resource": { - "resourceType": "Observation", - "id": "1073d9e6-4230-421f-95b2-d94e5cb6ba1b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:19c1b3c8-8e98-4c13-800d-31d529f7b5de", - "resource": { - "resourceType": "Observation", - "id": "19c1b3c8-8e98-4c13-800d-31d529f7b5de", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.22, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36497594-a72a-4914-977a-19c536b21d38", - "resource": { - "resourceType": "Procedure", - "id": "36497594-a72a-4914-977a-19c536b21d38", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "performedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7b813741-9057-40c5-b410-78bf1201a422", - "resource": { - "resourceType": "Procedure", - "id": "7b813741-9057-40c5-b410-78bf1201a422", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "performedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T22:58:39-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:08177039-c679-4659-b06f-cf18b1d5689f", - "resource": { - "resourceType": "MedicationRequest", - "id": "08177039-c679-4659-b06f-cf18b1d5689f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "authoredOn": "2016-01-03T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5ec76dfb-3152-4c4a-85ba-748687124fd8", - "resource": { - "resourceType": "Claim", - "id": "5ec76dfb-3152-4c4a-85ba-748687124fd8", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "created": "2016-01-03T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:08177039-c679-4659-b06f-cf18b1d5689f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - } - ] - } - ], - "total": { - "value": 20.82, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:172d8458-2b44-4b13-a99d-78e8ac972ae4", - "resource": { - "resourceType": "Immunization", - "id": "172d8458-2b44-4b13-a99d-78e8ac972ae4", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "occurrenceDateTime": "2016-01-03T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:38c19be3-9858-47af-a950-e6ad68dec814", - "resource": { - "resourceType": "DiagnosticReport", - "id": "38c19be3-9858-47af-a950-e6ad68dec814", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:00915ad1-e8da-44b4-b216-93153231b84a", - "display": "Glucose" - }, - { - "reference": "urn:uuid:ea0ab9c0-3864-4af7-9488-5750a22fe826", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:5687f0d4-bbd0-402c-84af-f91d4cce566f", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:f5f2abb5-10ad-4914-bb83-5f38c8e827f3", - "display": "Calcium" - }, - { - "reference": "urn:uuid:f762dfda-ffbf-4d97-8ae9-564213b6acb6", - "display": "Sodium" - }, - { - "reference": "urn:uuid:ff1ec2af-430d-42b9-b4bb-1ef21aab02f2", - "display": "Potassium" - }, - { - "reference": "urn:uuid:9b9877da-aad6-46b9-99e6-cc80aa86b1c8", - "display": "Chloride" - }, - { - "reference": "urn:uuid:493372ae-260b-412a-b3b7-461278f3cb77", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f38942dd-07f2-407a-846e-980b4f723dce", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f38942dd-07f2-407a-846e-980b4f723dce", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - }, - "effectiveDateTime": "2016-01-03T22:43:39-08:00", - "issued": "2016-01-03T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:5a61b2a3-bd85-421c-96c2-c6eb382ea08e", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:874a2c15-1c59-44de-8821-96d9f1e0a8da", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:df6499d7-8b55-43c4-b7df-efbdbcd67a09", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:d3a76fa9-be44-40d0-8ddc-a4b96a18a664", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:f5635958-2b38-4d0f-9de1-60956c7824dc", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:c9553d8a-a97c-449d-a201-7a8bde168a89", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:d6768ab4-a243-40fb-b6aa-923ea3154778", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:3155135a-faa8-44be-abd2-56e1490dded8", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:ebd1df5e-a263-42be-bc0f-7e5e80807537", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:fa4b4615-1eb8-429d-b92a-c39c2fb30fe3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:bc6a414e-d54e-4553-a22b-726d4786977d", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:5571a87e-7192-4b60-bf83-df6ab0fbefab", - "resource": { - "resourceType": "Claim", - "id": "5571a87e-7192-4b60-bf83-df6ab0fbefab", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "created": "2016-01-03T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:172d8458-2b44-4b13-a99d-78e8ac972ae4" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:36497594-a72a-4914-977a-19c536b21d38" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:7b813741-9057-40c5-b410-78bf1201a422" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 10455.00, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 920.83, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1d7e6885-3df9-46e7-a1cf-0c6241bfdfa8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1d7e6885-3df9-46e7-a1cf-0c6241bfdfa8", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5571a87e-7192-4b60-bf83-df6ab0fbefab" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2016-01-03T23:28:39-08:00", - "end": "2017-01-03T23:28:39-08:00" - }, - "created": "2016-01-03T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5571a87e-7192-4b60-bf83-df6ab0fbefab" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:8e3c7581-a34a-439c-8a9c-6bfea816ca4e" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 10455.00, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2091.0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 8364.0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 10455.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 10455.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-03T22:43:39-08:00", - "end": "2016-01-03T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 920.83, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 184.16600000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 736.6640000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 920.83, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 920.83, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 9213.08, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978", - "resource": { - "resourceType": "Encounter", - "id": "e295fa5c-4876-4211-9f98-cb19ffad4978", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:135cea4d-a4fe-45c1-b7e1-d408e00f6122", - "resource": { - "resourceType": "Observation", - "id": "135cea4d-a4fe-45c1-b7e1-d408e00f6122", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 73.821, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33e2a4d1-07d4-4166-90e3-8cfaa28490b5", - "resource": { - "resourceType": "Observation", - "id": "33e2a4d1-07d4-4166-90e3-8cfaa28490b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84f60f1d-17ba-440d-8772-6ca72cc268e7", - "resource": { - "resourceType": "Observation", - "id": "84f60f1d-17ba-440d-8772-6ca72cc268e7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:638f670b-877d-4b93-9602-1bd211c059ac", - "resource": { - "resourceType": "Observation", - "id": "638f670b-877d-4b93-9602-1bd211c059ac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d04b82d-ce65-4b38-9a2e-4b2538082481", - "resource": { - "resourceType": "Observation", - "id": "2d04b82d-ce65-4b38-9a2e-4b2538082481", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c441282e-8448-4720-8a59-70253c394b77", - "resource": { - "resourceType": "Observation", - "id": "c441282e-8448-4720-8a59-70253c394b77", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 109, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e2509d7-4970-426d-8750-11e4c3383459", - "resource": { - "resourceType": "Observation", - "id": "6e2509d7-4970-426d-8750-11e4c3383459", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 76, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a95e98ac-6ed4-4951-b024-db57012b857d", - "resource": { - "resourceType": "Observation", - "id": "a95e98ac-6ed4-4951-b024-db57012b857d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a15d3f24-b640-4d9e-93de-86ef97606a73", - "resource": { - "resourceType": "Observation", - "id": "a15d3f24-b640-4d9e-93de-86ef97606a73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 69.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8d87cd49-2f03-46aa-852b-fa391aeef5ff", - "resource": { - "resourceType": "Observation", - "id": "8d87cd49-2f03-46aa-852b-fa391aeef5ff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 8.57, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:608a1fff-c93a-457e-b09c-3930d8a0dbc4", - "resource": { - "resourceType": "Observation", - "id": "608a1fff-c93a-457e-b09c-3930d8a0dbc4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac885c3a-9ac0-45e2-a9fa-88b683075c36", - "resource": { - "resourceType": "Observation", - "id": "ac885c3a-9ac0-45e2-a9fa-88b683075c36", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.27, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a435ae1-1fe9-4903-9631-a024c8234d17", - "resource": { - "resourceType": "Observation", - "id": "3a435ae1-1fe9-4903-9631-a024c8234d17", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 138.65, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e718f30a-6320-4c98-a49d-fdca25622adc", - "resource": { - "resourceType": "Observation", - "id": "e718f30a-6320-4c98-a49d-fdca25622adc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 3.96, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a8b3db26-2054-4703-9e9e-2f2095dc5316", - "resource": { - "resourceType": "Observation", - "id": "a8b3db26-2054-4703-9e9e-2f2095dc5316", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 106.11, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ea580ec-d735-479b-bfb9-85b64502d269", - "resource": { - "resourceType": "Observation", - "id": "2ea580ec-d735-479b-bfb9-85b64502d269", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 27.24, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", - "resource": { - "resourceType": "Observation", - "id": "6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 196.37, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9db44093-f9ca-40f7-b967-b222f6bba772", - "resource": { - "resourceType": "Observation", - "id": "9db44093-f9ca-40f7-b967-b222f6bba772", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 117.3, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:761a3bc8-ee20-4973-9139-dc7ef7141ea9", - "resource": { - "resourceType": "Observation", - "id": "761a3bc8-ee20-4973-9139-dc7ef7141ea9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 94.29, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3389b7f7-2a0c-4020-bb3d-161915a39a85", - "resource": { - "resourceType": "Observation", - "id": "3389b7f7-2a0c-4020-bb3d-161915a39a85", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.62, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:01c2b0d1-7b98-47c2-ae5d-db0e1ef31c18", - "resource": { - "resourceType": "Observation", - "id": "01c2b0d1-7b98-47c2-ae5d-db0e1ef31c18", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d31c6951-6ee5-4053-bb36-ee817b78a05a", - "resource": { - "resourceType": "Observation", - "id": "d31c6951-6ee5-4053-bb36-ee817b78a05a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "valueQuantity": { - "value": 5.9, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b1709e49-df90-48c5-ab67-caf89ada6aa2", - "resource": { - "resourceType": "Procedure", - "id": "b1709e49-df90-48c5-ab67-caf89ada6aa2", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "performedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:cd85bf62-5a17-44f2-ae66-44f7d7a5f379", - "resource": { - "resourceType": "Procedure", - "id": "cd85bf62-5a17-44f2-ae66-44f7d7a5f379", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "performedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T22:58:39-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4f446f5e-a358-449a-b1ae-f5a7b089f321", - "resource": { - "resourceType": "MedicationRequest", - "id": "4f446f5e-a358-449a-b1ae-f5a7b089f321", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "authoredOn": "2017-01-08T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:e888b203-22af-447e-9d74-f0893d521b83", - "resource": { - "resourceType": "Claim", - "id": "e888b203-22af-447e-9d74-f0893d521b83", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "created": "2017-01-08T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:4f446f5e-a358-449a-b1ae-f5a7b089f321" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - } - ] - } - ], - "total": { - "value": 10.09, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7041f895-b10b-47f4-992f-f46aa1e0ea67", - "resource": { - "resourceType": "Immunization", - "id": "7041f895-b10b-47f4-992f-f46aa1e0ea67", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "occurrenceDateTime": "2017-01-08T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a550bc16-c06b-4ba1-95ae-7fc33936ebf4", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a550bc16-c06b-4ba1-95ae-7fc33936ebf4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:a15d3f24-b640-4d9e-93de-86ef97606a73", - "display": "Glucose" - }, - { - "reference": "urn:uuid:8d87cd49-2f03-46aa-852b-fa391aeef5ff", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:608a1fff-c93a-457e-b09c-3930d8a0dbc4", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:ac885c3a-9ac0-45e2-a9fa-88b683075c36", - "display": "Calcium" - }, - { - "reference": "urn:uuid:3a435ae1-1fe9-4903-9631-a024c8234d17", - "display": "Sodium" - }, - { - "reference": "urn:uuid:e718f30a-6320-4c98-a49d-fdca25622adc", - "display": "Potassium" - }, - { - "reference": "urn:uuid:a8b3db26-2054-4703-9e9e-2f2095dc5316", - "display": "Chloride" - }, - { - "reference": "urn:uuid:2ea580ec-d735-479b-bfb9-85b64502d269", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7070af55-59a8-4f27-bacd-238f8f9cce37", - "resource": { - "resourceType": "DiagnosticReport", - "id": "7070af55-59a8-4f27-bacd-238f8f9cce37", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - }, - "effectiveDateTime": "2017-01-08T22:43:39-08:00", - "issued": "2017-01-08T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:6e1c3077-2ec0-4304-a0b2-6cf2d4a47350", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:9db44093-f9ca-40f7-b967-b222f6bba772", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:761a3bc8-ee20-4973-9139-dc7ef7141ea9", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:3389b7f7-2a0c-4020-bb3d-161915a39a85", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5", - "resource": { - "resourceType": "Claim", - "id": "43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "created": "2017-01-08T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7041f895-b10b-47f4-992f-f46aa1e0ea67" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b1709e49-df90-48c5-ab67-caf89ada6aa2" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:cd85bf62-5a17-44f2-ae66-44f7d7a5f379" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 13413.48, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 556.50, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:46e70939-620c-4560-9f97-3e55f463bf2b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "46e70939-620c-4560-9f97-3e55f463bf2b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2017-01-08T23:28:39-08:00", - "end": "2018-01-08T23:28:39-08:00" - }, - "created": "2017-01-08T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:43a3cab0-25ad-4cd7-979d-8b8dfb25c9b5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e295fa5c-4876-4211-9f98-cb19ffad4978" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 13413.48, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2682.696, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 10730.784, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 13413.48, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 13413.48, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-08T22:43:39-08:00", - "end": "2017-01-08T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 556.50, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 111.30000000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 445.20000000000005, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 556.50, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 556.50, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 11288.4, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f", - "resource": { - "resourceType": "Encounter", - "id": "a798d69d-3188-4bdd-b0c7-22346acdf71f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b7ddb3cf-607f-4ab7-9f5a-d1180f3eef71", - "resource": { - "resourceType": "Observation", - "id": "b7ddb3cf-607f-4ab7-9f5a-d1180f3eef71", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 77.513, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d00b5f6-c3f8-471f-a9d4-7a4b04de4a66", - "resource": { - "resourceType": "Observation", - "id": "9d00b5f6-c3f8-471f-a9d4-7a4b04de4a66", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b240bf5c-67bb-42dd-8ca9-8734d02a3f55", - "resource": { - "resourceType": "Observation", - "id": "b240bf5c-67bb-42dd-8ca9-8734d02a3f55", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ea64c02-cafa-4e00-a0cc-ce889e8ba78b", - "resource": { - "resourceType": "Observation", - "id": "1ea64c02-cafa-4e00-a0cc-ce889e8ba78b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ab7b0cc-a56e-44fe-8589-c60f4e19b42d", - "resource": { - "resourceType": "Observation", - "id": "1ab7b0cc-a56e-44fe-8589-c60f4e19b42d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dec2c45a-3aaf-4189-a1f2-a22e79f671fc", - "resource": { - "resourceType": "Observation", - "id": "dec2c45a-3aaf-4189-a1f2-a22e79f671fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 130, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6339544d-a171-4a3d-a199-9707ab115717", - "resource": { - "resourceType": "Observation", - "id": "6339544d-a171-4a3d-a199-9707ab115717", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8ff42d2a-d512-4530-a8f2-c96c20dc306f", - "resource": { - "resourceType": "Observation", - "id": "8ff42d2a-d512-4530-a8f2-c96c20dc306f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3b617d2-944a-4e8d-9869-3796c1c9fd9a", - "resource": { - "resourceType": "Observation", - "id": "f3b617d2-944a-4e8d-9869-3796c1c9fd9a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 74.49, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aa78d1db-48e2-4fec-a765-02225e935783", - "resource": { - "resourceType": "Observation", - "id": "aa78d1db-48e2-4fec-a765-02225e935783", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 14.7, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", - "resource": { - "resourceType": "Observation", - "id": "a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.77, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:21fb774e-4614-46ed-b4a9-585f8c9f3116", - "resource": { - "resourceType": "Observation", - "id": "21fb774e-4614-46ed-b4a9-585f8c9f3116", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.62, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", - "resource": { - "resourceType": "Observation", - "id": "8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 143.53, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66005231-6d08-40fb-acc0-117a6333d101", - "resource": { - "resourceType": "Observation", - "id": "66005231-6d08-40fb-acc0-117a6333d101", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.86, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:487b2d25-a627-4de2-8dff-b73260d85de9", - "resource": { - "resourceType": "Observation", - "id": "487b2d25-a627-4de2-8dff-b73260d85de9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 110.58, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a9fffc1-fbae-4c6c-a9f8-1e459515239d", - "resource": { - "resourceType": "Observation", - "id": "8a9fffc1-fbae-4c6c-a9f8-1e459515239d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 25.74, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b27f8303-1b2c-4e9c-b23b-4fa4d723517c", - "resource": { - "resourceType": "Observation", - "id": "b27f8303-1b2c-4e9c-b23b-4fa4d723517c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:18141f56-2611-4b5e-b81a-3c59ad9995bd", - "resource": { - "resourceType": "Observation", - "id": "18141f56-2611-4b5e-b81a-3c59ad9995bd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "valueQuantity": { - "value": 5.8, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649", - "resource": { - "resourceType": "Procedure", - "id": "4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "performedPeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7a1740bb-9fed-48bf-8291-8034428337be", - "resource": { - "resourceType": "MedicationRequest", - "id": "7a1740bb-9fed-48bf-8291-8034428337be", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "authoredOn": "2018-01-14T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:521b3337-f168-4338-a7cc-97fedde9be21", - "resource": { - "resourceType": "Claim", - "id": "521b3337-f168-4338-a7cc-97fedde9be21", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "created": "2018-01-14T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:7a1740bb-9fed-48bf-8291-8034428337be" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - } - ] - } - ], - "total": { - "value": 35.14, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:523a74d7-7cb0-4d7e-8796-b6069eb4f254", - "resource": { - "resourceType": "Immunization", - "id": "523a74d7-7cb0-4d7e-8796-b6069eb4f254", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "occurrenceDateTime": "2018-01-14T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b2473f69-54db-4e35-80d5-e8dbf484f577", - "resource": { - "resourceType": "DiagnosticReport", - "id": "b2473f69-54db-4e35-80d5-e8dbf484f577", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - }, - "effectiveDateTime": "2018-01-14T22:43:39-08:00", - "issued": "2018-01-14T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:f3b617d2-944a-4e8d-9869-3796c1c9fd9a", - "display": "Glucose" - }, - { - "reference": "urn:uuid:aa78d1db-48e2-4fec-a765-02225e935783", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:a1076e80-6ea9-4e7e-8ad5-627b91eb24ab", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:21fb774e-4614-46ed-b4a9-585f8c9f3116", - "display": "Calcium" - }, - { - "reference": "urn:uuid:8e5f8e46-18db-4fd2-aac7-9ade98e3a1c2", - "display": "Sodium" - }, - { - "reference": "urn:uuid:66005231-6d08-40fb-acc0-117a6333d101", - "display": "Potassium" - }, - { - "reference": "urn:uuid:487b2d25-a627-4de2-8dff-b73260d85de9", - "display": "Chloride" - }, - { - "reference": "urn:uuid:8a9fffc1-fbae-4c6c-a9f8-1e459515239d", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:5807b9f9-621d-4c1b-81bc-539091b3ec86", - "resource": { - "resourceType": "Claim", - "id": "5807b9f9-621d-4c1b-81bc-539091b3ec86", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "created": "2018-01-14T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:523a74d7-7cb0-4d7e-8796-b6069eb4f254" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4b2a8ee8-0f9b-43e2-96b8-fd11a4da3649" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 9408.99, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:70312de5-f3e0-49f7-ab70-80f5554196d0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "70312de5-f3e0-49f7-ab70-80f5554196d0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5807b9f9-621d-4c1b-81bc-539091b3ec86" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2018-01-14T23:13:39-08:00", - "end": "2019-01-14T23:13:39-08:00" - }, - "created": "2018-01-14T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5807b9f9-621d-4c1b-81bc-539091b3ec86" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a798d69d-3188-4bdd-b0c7-22346acdf71f" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2018-01-14T22:43:39-08:00", - "end": "2018-01-14T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 9408.99, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1881.798, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7527.192, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 9408.99, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 9408.99, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7639.608, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2", - "resource": { - "resourceType": "Encounter", - "id": "64e01d2a-98d3-4934-9447-c27f361a15b2", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:26a97f52-9906-41b5-ac58-c30e636f82c0", - "resource": { - "resourceType": "Observation", - "id": "26a97f52-9906-41b5-ac58-c30e636f82c0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 54.986, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:edaf0e70-edc4-4f92-b3e1-e547ad28742b", - "resource": { - "resourceType": "Observation", - "id": "edaf0e70-edc4-4f92-b3e1-e547ad28742b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d60c87e8-182b-4ab1-bdeb-da169a5a5a76", - "resource": { - "resourceType": "Observation", - "id": "d60c87e8-182b-4ab1-bdeb-da169a5a5a76", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c970e1a-bb68-4c47-bc2d-28d0d0c8d938", - "resource": { - "resourceType": "Observation", - "id": "4c970e1a-bb68-4c47-bc2d-28d0d0c8d938", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:72dd89a6-49fd-46a7-b743-0c8f5f77b2ea", - "resource": { - "resourceType": "Observation", - "id": "72dd89a6-49fd-46a7-b743-0c8f5f77b2ea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b58f2406-ac64-4a5e-b703-b766fce60610", - "resource": { - "resourceType": "Observation", - "id": "b58f2406-ac64-4a5e-b703-b766fce60610", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 128, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:56946b5c-f75a-4a75-a5fb-daf51ad8a248", - "resource": { - "resourceType": "Observation", - "id": "56946b5c-f75a-4a75-a5fb-daf51ad8a248", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 98, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ad1591a1-778c-4eeb-b91b-93e546e6eb62", - "resource": { - "resourceType": "Observation", - "id": "ad1591a1-778c-4eeb-b91b-93e546e6eb62", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c8508776-0eec-4986-8d11-43ff06db4ec7", - "resource": { - "resourceType": "Observation", - "id": "c8508776-0eec-4986-8d11-43ff06db4ec7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 71.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", - "resource": { - "resourceType": "Observation", - "id": "0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 12.21, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30c82479-f6c5-4488-81be-cc60d82777b2", - "resource": { - "resourceType": "Observation", - "id": "30c82479-f6c5-4488-81be-cc60d82777b2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 1.33, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c71e1520-6255-4b1d-a642-67924c2e360b", - "resource": { - "resourceType": "Observation", - "id": "c71e1520-6255-4b1d-a642-67924c2e360b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.66, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47ef4e37-42fe-476f-b391-8b090bb5c98f", - "resource": { - "resourceType": "Observation", - "id": "47ef4e37-42fe-476f-b391-8b090bb5c98f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 140.36, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:835f2331-8c39-4c91-a24b-62932edd109b", - "resource": { - "resourceType": "Observation", - "id": "835f2331-8c39-4c91-a24b-62932edd109b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 4.49, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2004b881-cff4-4400-9767-ce1b4f1110ba", - "resource": { - "resourceType": "Observation", - "id": "2004b881-cff4-4400-9767-ce1b4f1110ba", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 103.84, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9904170d-b63f-48db-839b-63948bdab2f9", - "resource": { - "resourceType": "Observation", - "id": "9904170d-b63f-48db-839b-63948bdab2f9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 28.46, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0eeb5a4-8d9a-4855-934a-7273468a2eee", - "resource": { - "resourceType": "Observation", - "id": "b0eeb5a4-8d9a-4855-934a-7273468a2eee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a4ee721-c6c6-43e9-888e-f7ad49fc682d", - "resource": { - "resourceType": "Observation", - "id": "5a4ee721-c6c6-43e9-888e-f7ad49fc682d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "valueQuantity": { - "value": 5.81, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0a7424cc-ecae-4092-9620-2115372fb20b", - "resource": { - "resourceType": "Procedure", - "id": "0a7424cc-ecae-4092-9620-2115372fb20b", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "performedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b463ec82-b360-4639-b151-c3b8a369ab89", - "resource": { - "resourceType": "Procedure", - "id": "b463ec82-b360-4639-b151-c3b8a369ab89", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "performedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T22:58:39-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:cc39fe6e-11cd-483c-a66f-89ec15a72485", - "resource": { - "resourceType": "MedicationRequest", - "id": "cc39fe6e-11cd-483c-a66f-89ec15a72485", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "authoredOn": "2019-01-20T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:31313185-8995-4f14-9b0d-811fb8d9020a", - "resource": { - "resourceType": "Claim", - "id": "31313185-8995-4f14-9b0d-811fb8d9020a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "created": "2019-01-20T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:cc39fe6e-11cd-483c-a66f-89ec15a72485" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - } - ] - } - ], - "total": { - "value": 34.72, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:54e00fdd-e368-4c6e-9335-b1001aa3ce96", - "resource": { - "resourceType": "Immunization", - "id": "54e00fdd-e368-4c6e-9335-b1001aa3ce96", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "occurrenceDateTime": "2019-01-20T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a635116e-7100-454a-aacf-7ecfee9f5e47", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a635116e-7100-454a-aacf-7ecfee9f5e47", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - }, - "effectiveDateTime": "2019-01-20T22:43:39-08:00", - "issued": "2019-01-20T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:c8508776-0eec-4986-8d11-43ff06db4ec7", - "display": "Glucose" - }, - { - "reference": "urn:uuid:0e42ca44-52cc-4d20-a7ba-3aadf7ab840d", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:30c82479-f6c5-4488-81be-cc60d82777b2", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:c71e1520-6255-4b1d-a642-67924c2e360b", - "display": "Calcium" - }, - { - "reference": "urn:uuid:47ef4e37-42fe-476f-b391-8b090bb5c98f", - "display": "Sodium" - }, - { - "reference": "urn:uuid:835f2331-8c39-4c91-a24b-62932edd109b", - "display": "Potassium" - }, - { - "reference": "urn:uuid:2004b881-cff4-4400-9767-ce1b4f1110ba", - "display": "Chloride" - }, - { - "reference": "urn:uuid:9904170d-b63f-48db-839b-63948bdab2f9", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:8c44ac1b-9c63-485b-b706-da3bf7b05ce7", - "resource": { - "resourceType": "Claim", - "id": "8c44ac1b-9c63-485b-b706-da3bf7b05ce7", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "created": "2019-01-20T23:28:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:54e00fdd-e368-4c6e-9335-b1001aa3ce96" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:0a7424cc-ecae-4092-9620-2115372fb20b" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:b463ec82-b360-4639-b151-c3b8a369ab89" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 7705.04, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 553.63, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:03b8f82d-9d61-4703-87f3-42089f7b0444", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "03b8f82d-9d61-4703-87f3-42089f7b0444", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8c44ac1b-9c63-485b-b706-da3bf7b05ce7" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2019-01-20T23:28:39-08:00", - "end": "2020-01-20T23:28:39-08:00" - }, - "created": "2019-01-20T23:28:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8c44ac1b-9c63-485b-b706-da3bf7b05ce7" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:64e01d2a-98d3-4934-9447-c27f361a15b2" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 7705.04, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1541.008, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 6164.032, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 7705.04, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 7705.04, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2019-01-20T22:43:39-08:00", - "end": "2019-01-20T23:28:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 553.63, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 110.726, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 442.904, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 553.63, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 553.63, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 6719.352000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0", - "resource": { - "resourceType": "Encounter", - "id": "fc0f7e2d-938d-4f37-8f1f-726e614bb0d0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Mrs. Lorette239 Marvin195" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "individual": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - } - } - ], - "period": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4ed1bb6d-ea9d-43da-92d4-f881780ba67c", - "resource": { - "resourceType": "Observation", - "id": "4ed1bb6d-ea9d-43da-92d4-f881780ba67c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "19926-5", - "display": "FEV1/FVC" - } - ], - "text": "FEV1/FVC" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 74.344, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6abfde09-c630-43b7-8e14-1573dd652ae7", - "resource": { - "resourceType": "Observation", - "id": "6abfde09-c630-43b7-8e14-1573dd652ae7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 161, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bd5fa694-5a73-4cc2-8c98-ef7573efe2dc", - "resource": { - "resourceType": "Observation", - "id": "bd5fa694-5a73-4cc2-8c98-ef7573efe2dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c3f55167-2ae7-412e-b9d3-b713903b22d9", - "resource": { - "resourceType": "Observation", - "id": "c3f55167-2ae7-412e-b9d3-b713903b22d9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:09c2a3f5-cd18-4bf1-9e17-f26dff936459", - "resource": { - "resourceType": "Observation", - "id": "09c2a3f5-cd18-4bf1-9e17-f26dff936459", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 30.4, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1af95c19-cd46-4bfc-9150-b417cd8a05de", - "resource": { - "resourceType": "Observation", - "id": "1af95c19-cd46-4bfc-9150-b417cd8a05de", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 139, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c7d87b7-088a-4fba-8c26-7255530ceabd", - "resource": { - "resourceType": "Observation", - "id": "3c7d87b7-088a-4fba-8c26-7255530ceabd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 86, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac1f5163-1e12-4dce-97cd-8dc303617416", - "resource": { - "resourceType": "Observation", - "id": "ac1f5163-1e12-4dce-97cd-8dc303617416", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d753d661-2633-4635-b0d2-ef8fdcbcfdfc", - "resource": { - "resourceType": "Observation", - "id": "d753d661-2633-4635-b0d2-ef8fdcbcfdfc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 74.49, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f21625c-79c1-466c-af0c-1d2ee5e1ff43", - "resource": { - "resourceType": "Observation", - "id": "5f21625c-79c1-466c-af0c-1d2ee5e1ff43", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 16.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7542f0dd-48c7-4d55-aaee-af92a2a21a5e", - "resource": { - "resourceType": "Observation", - "id": "7542f0dd-48c7-4d55-aaee-af92a2a21a5e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 0.96, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d95b95e9-82c7-4384-8dfc-89983af44610", - "resource": { - "resourceType": "Observation", - "id": "d95b95e9-82c7-4384-8dfc-89983af44610", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 9.12, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", - "resource": { - "resourceType": "Observation", - "id": "86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 141.99, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6866baa3-d56e-4e8a-b32f-a47b505acfd3", - "resource": { - "resourceType": "Observation", - "id": "6866baa3-d56e-4e8a-b32f-a47b505acfd3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 3.92, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:795f464e-3921-4b38-a87d-72529e633ee2", - "resource": { - "resourceType": "Observation", - "id": "795f464e-3921-4b38-a87d-72529e633ee2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 103.19, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51bb4bb3-94bb-408c-89d5-e470b389dbc8", - "resource": { - "resourceType": "Observation", - "id": "51bb4bb3-94bb-408c-89d5-e470b389dbc8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 27.47, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:af1547cb-5908-4d07-bac1-0bf1eb9d295d", - "resource": { - "resourceType": "Observation", - "id": "af1547cb-5908-4d07-bac1-0bf1eb9d295d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 189.91, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76ba1a3f-068b-491b-82a4-21f5e4c30e74", - "resource": { - "resourceType": "Observation", - "id": "76ba1a3f-068b-491b-82a4-21f5e4c30e74", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 116.92, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9c550c5-54da-459e-bb00-81a27fc89390", - "resource": { - "resourceType": "Observation", - "id": "a9c550c5-54da-459e-bb00-81a27fc89390", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 87.95, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f96acc88-5535-41be-a4c3-17a762512090", - "resource": { - "resourceType": "Observation", - "id": "f96acc88-5535-41be-a4c3-17a762512090", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 78.57, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82b37be7-5742-4ec7-9ba2-e3b5b8e4930f", - "resource": { - "resourceType": "Observation", - "id": "82b37be7-5742-4ec7-9ba2-e3b5b8e4930f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "8517006", - "display": "Former smoker" - } - ], - "text": "Former smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd2afe05-4b40-46f6-99f1-de9b9d836621", - "resource": { - "resourceType": "Observation", - "id": "dd2afe05-4b40-46f6-99f1-de9b9d836621", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "valueQuantity": { - "value": 6.19, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d479c20-61eb-4cf2-b3cb-a03aa7a276fb", - "resource": { - "resourceType": "Procedure", - "id": "7d479c20-61eb-4cf2-b3cb-a03aa7a276fb", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "performedPeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T22:58:39-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1", - "display": "Pulmonary emphysema (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf", - "resource": { - "resourceType": "MedicationRequest", - "id": "4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "896209", - "display": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - } - ], - "text": "60 ACTUAT Fluticasone propionate 0.25 MG/ACTUAT / salmeterol 0.05 MG/ACTUAT Dry Powder Inhaler" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "authoredOn": "2020-01-26T22:43:39-08:00", - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "display": "Dr. Cheryle584 Koelpin146" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f600d65e-ba3c-4b9f-86e2-2fe813bcc8a1" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:970138e8-c02f-4600-bd18-60b60d9c837f", - "resource": { - "resourceType": "Claim", - "id": "970138e8-c02f-4600-bd18-60b60d9c837f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "created": "2020-01-26T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:4e7a5f47-9aa9-44bf-8ce0-60d3ce64d5bf" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - } - ] - } - ], - "total": { - "value": 17.59, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:140cb569-a32b-4b6b-abc3-85157b9ee101", - "resource": { - "resourceType": "Immunization", - "id": "140cb569-a32b-4b6b-abc3-85157b9ee101", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "occurrenceDateTime": "2020-01-26T22:43:39-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c1dcfdd5-7c2d-4654-94f2-9eaaa22137a7", - "resource": { - "resourceType": "DiagnosticReport", - "id": "c1dcfdd5-7c2d-4654-94f2-9eaaa22137a7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:d753d661-2633-4635-b0d2-ef8fdcbcfdfc", - "display": "Glucose" - }, - { - "reference": "urn:uuid:5f21625c-79c1-466c-af0c-1d2ee5e1ff43", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:7542f0dd-48c7-4d55-aaee-af92a2a21a5e", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:d95b95e9-82c7-4384-8dfc-89983af44610", - "display": "Calcium" - }, - { - "reference": "urn:uuid:86cbd7b5-d627-44a4-bcf6-1dbc7c2a1368", - "display": "Sodium" - }, - { - "reference": "urn:uuid:6866baa3-d56e-4e8a-b32f-a47b505acfd3", - "display": "Potassium" - }, - { - "reference": "urn:uuid:795f464e-3921-4b38-a87d-72529e633ee2", - "display": "Chloride" - }, - { - "reference": "urn:uuid:51bb4bb3-94bb-408c-89d5-e470b389dbc8", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f1737ab8-8d19-436f-aea9-a010acce57da", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f1737ab8-8d19-436f-aea9-a010acce57da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "encounter": { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - }, - "effectiveDateTime": "2020-01-26T22:43:39-08:00", - "issued": "2020-01-26T22:43:39.914-08:00", - "result": [ - { - "reference": "urn:uuid:af1547cb-5908-4d07-bac1-0bf1eb9d295d", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:76ba1a3f-068b-491b-82a4-21f5e4c30e74", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:a9c550c5-54da-459e-bb00-81a27fc89390", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:f96acc88-5535-41be-a4c3-17a762512090", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:40556fd6-be95-44a1-beeb-bed2a8cb1513", - "resource": { - "resourceType": "Claim", - "id": "40556fd6-be95-44a1-beeb-bed2a8cb1513", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b", - "display": "Lorette239 Marvin195" - }, - "billablePeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "created": "2020-01-26T23:13:39-08:00", - "provider": { - "reference": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "display": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC." - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:140cb569-a32b-4b6b-abc3-85157b9ee101" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7d479c20-61eb-4cf2-b3cb-a03aa7a276fb" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "net": { - "value": 14224.84, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8f1b7b95-c113-48fc-a6a4-b73187564ffe", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8f1b7b95-c113-48fc-a6a4-b73187564ffe", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "requester": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "performer": [ - { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "40556fd6-be95-44a1-beeb-bed2a8cb1513" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:af0e4a42-a1ef-4c2f-ad31-9338a8b8fb7b" - }, - "billablePeriod": { - "start": "2020-01-26T23:13:39-08:00", - "end": "2021-01-26T23:13:39-08:00" - }, - "created": "2020-01-26T23:13:39-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:40556fd6-be95-44a1-beeb-bed2a8cb1513" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:fc0f7e2d-938d-4f37-8f1f-726e614bb0d0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "127783003", - "display": "Spirometry (procedure)" - } - ], - "text": "Spirometry (procedure)" - }, - "servicedPeriod": { - "start": "2020-01-26T22:43:39-08:00", - "end": "2020-01-26T23:13:39-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 14224.84, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2844.9680000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 11379.872000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 14224.84, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 14224.84, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 11492.288, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json deleted file mode 100644 index a3a593a48dea..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Norberto865_Cole117_a5e08ef9-fd71-4273-a53a-d5f5df7926f4.json +++ /dev/null @@ -1,16971 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "resource": { - "resourceType": "Patient", - "id": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -7883787131534806314 Population seed: 1586368870505
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Cinthia596 Thompson596" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", - "valueAddress": { - "city": "Quincy", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.0 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 16.0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "SS", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-52-7917" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99926136" - } - ], - "name": [ - { - "use": "official", - "family": "Cole117", - "given": [ - "Norberto865" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-257-6498", - "use": "home" - } - ], - "gender": "male", - "birthDate": "2003-12-29", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 41.64238723329318 - }, - { - "url": "longitude", - "valueDecimal": -71.16534881278795 - } - ] - } - ], - "line": [ - "436 Rogahn Alley" - ], - "city": "Fall River", - "state": "Massachusetts", - "postalCode": "02747", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "resource": { - "resourceType": "Organization", - "id": "75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "75bdb2ec-cbd0-35af-b0fe-18b111890b67" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP235727", - "address": [ - { - "line": [ - "67 SLADES FERRY BLVD" - ], - "city": "SOMERSET", - "state": "MA", - "postalCode": "02726", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "resource": { - "resourceType": "Practitioner", - "id": "b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999931859" - } - ], - "active": true, - "name": [ - { - "family": "Mante251", - "given": [ - "Bertie593" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Bertie593.Mante251@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "67 SLADES FERRY BLVD" - ], - "city": "SOMERSET", - "state": "MA", - "postalCode": "02726", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86", - "resource": { - "resourceType": "Encounter", - "id": "3c8db206-734b-4a90-a8bd-c091a117dc86", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2010-12-20T08:41:37-08:00", - "end": "2010-12-20T08:56:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2010-12-20T08:41:37-08:00", - "end": "2010-12-20T08:56:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:23a2b208-2135-4a6c-885c-e246549a3f8d", - "resource": { - "resourceType": "Observation", - "id": "23a2b208-2135-4a6c-885c-e246549a3f8d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 119.1, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e88cac97-70b3-4ec2-bcf4-a1c493d4cdc3", - "resource": { - "resourceType": "Observation", - "id": "e88cac97-70b3-4ec2-bcf4-a1c493d4cdc3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3125fbf8-307d-43a9-bb38-409595e448cc", - "resource": { - "resourceType": "Observation", - "id": "3125fbf8-307d-43a9-bb38-409595e448cc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 26.9, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35d44b26-697d-4eb0-b661-44a50a369a16", - "resource": { - "resourceType": "Observation", - "id": "35d44b26-697d-4eb0-b661-44a50a369a16", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 18.96, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67c2496e-7394-40c1-ae13-a7614305fe0a", - "resource": { - "resourceType": "Observation", - "id": "67c2496e-7394-40c1-ae13-a7614305fe0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 94.625, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92be6692-adf0-49ea-abf1-447192a3333d", - "resource": { - "resourceType": "Observation", - "id": "92be6692-adf0-49ea-abf1-447192a3333d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e5638cb4-8a26-4446-b6e5-100fe98d5166", - "resource": { - "resourceType": "Observation", - "id": "e5638cb4-8a26-4446-b6e5-100fe98d5166", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 73, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ec688fc5-cbf0-43cd-a7e4-86be00621b30", - "resource": { - "resourceType": "Observation", - "id": "ec688fc5-cbf0-43cd-a7e4-86be00621b30", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:097893e3-f0e0-4cb9-ac46-18f37b9dc94a", - "resource": { - "resourceType": "Observation", - "id": "097893e3-f0e0-4cb9-ac46-18f37b9dc94a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "effectiveDateTime": "2010-12-20T08:41:37-08:00", - "issued": "2010-12-20T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4", - "resource": { - "resourceType": "Immunization", - "id": "1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - }, - "occurrenceDateTime": "2010-12-20T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4b322488-0670-4432-8115-e4853cda7d23", - "resource": { - "resourceType": "Claim", - "id": "4b322488-0670-4432-8115-e4853cda7d23", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2010-12-20T08:41:37-08:00", - "end": "2010-12-20T08:56:37-08:00" - }, - "created": "2010-12-20T08:56:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1cf87ca8-4e6d-42d6-9a0a-ad38cfdee2c4" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:238fb15a-a86f-40a9-acb8-b9b801234ac2", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "238fb15a-a86f-40a9-acb8-b9b801234ac2", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4b322488-0670-4432-8115-e4853cda7d23" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2010-12-20T08:56:37-08:00", - "end": "2011-12-20T08:56:37-08:00" - }, - "created": "2010-12-20T08:56:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4b322488-0670-4432-8115-e4853cda7d23" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2010-12-20T08:41:37-08:00", - "end": "2010-12-20T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3c8db206-734b-4a90-a8bd-c091a117dc86" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2010-12-20T08:41:37-08:00", - "end": "2010-12-20T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b", - "resource": { - "resourceType": "Encounter", - "id": "77756f23-e4c6-4726-876a-df37611c202b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0902d9d3-1ae2-4240-b2ba-9df1e1e31b38", - "resource": { - "resourceType": "Observation", - "id": "0902d9d3-1ae2-4240-b2ba-9df1e1e31b38", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 125.1, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4301ad0f-7cfc-4a73-8445-607a0b01388a", - "resource": { - "resourceType": "Observation", - "id": "4301ad0f-7cfc-4a73-8445-607a0b01388a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51d6cc0f-ffd3-44f7-b4ac-0a17a252dfdf", - "resource": { - "resourceType": "Observation", - "id": "51d6cc0f-ffd3-44f7-b4ac-0a17a252dfdf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 30.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3792f8fc-4c29-4f60-bceb-f0940eb4be32", - "resource": { - "resourceType": "Observation", - "id": "3792f8fc-4c29-4f60-bceb-f0940eb4be32", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 19.51, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7b707ea1-3ac8-427a-80d7-d21da0351a40", - "resource": { - "resourceType": "Observation", - "id": "7b707ea1-3ac8-427a-80d7-d21da0351a40", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 93.686, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3208461d-688e-471d-90b7-d456109d073f", - "resource": { - "resourceType": "Observation", - "id": "3208461d-688e-471d-90b7-d456109d073f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 124, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7973049-699a-413c-8f00-b52fc1ecac6e", - "resource": { - "resourceType": "Observation", - "id": "b7973049-699a-413c-8f00-b52fc1ecac6e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 90, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d8bd8eb-4404-460a-8f37-d9016facedec", - "resource": { - "resourceType": "Observation", - "id": "7d8bd8eb-4404-460a-8f37-d9016facedec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1ccae92-2e8d-4ca4-a47e-53e8a684de26", - "resource": { - "resourceType": "Observation", - "id": "f1ccae92-2e8d-4ca4-a47e-53e8a684de26", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "effectiveDateTime": "2011-12-26T08:41:37-08:00", - "issued": "2011-12-26T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6bb3e397-48b3-4830-94e8-626289c8279e", - "resource": { - "resourceType": "Procedure", - "id": "6bb3e397-48b3-4830-94e8-626289c8279e", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "performedPeriod": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T08:56:37-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4a67f3d4-0e39-4a6c-996e-2235dbdc6674", - "resource": { - "resourceType": "Immunization", - "id": "4a67f3d4-0e39-4a6c-996e-2235dbdc6674", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - }, - "occurrenceDateTime": "2011-12-26T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54", - "resource": { - "resourceType": "Claim", - "id": "4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "created": "2011-12-26T09:11:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4a67f3d4-0e39-4a6c-996e-2235dbdc6674" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:6bb3e397-48b3-4830-94e8-626289c8279e" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 970.60, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ca620d16-0e4d-4829-8726-48661c7b88d6", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ca620d16-0e4d-4829-8726-48661c7b88d6", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2011-12-26T09:11:37-08:00", - "end": "2012-12-26T09:11:37-08:00" - }, - "created": "2011-12-26T09:11:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4940c0b8-6b56-4a2a-9843-6ffbb8fa5b54" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:77756f23-e4c6-4726-876a-df37611c202b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2011-12-26T08:41:37-08:00", - "end": "2011-12-26T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 970.60, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 194.12, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 776.48, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 970.60, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 970.60, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 888.8960000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "resource": { - "resourceType": "Organization", - "id": "23834663-ed53-3da9-b330-d6e1ecb8428e", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "23834663-ed53-3da9-b330-d6e1ecb8428e" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SOUTHCOAST HOSPITAL GROUP, INC", - "telecom": [ - { - "system": "phone", - "value": "5086793131" - } - ], - "address": [ - { - "line": [ - "363 HIGHLAND AVENUE" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02720", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "resource": { - "resourceType": "Practitioner", - "id": "f411d1de-1d0c-365f-8a71-8d225cb962af", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999699" - } - ], - "active": true, - "name": [ - { - "family": "Brakus656", - "given": [ - "Gillian484" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Gillian484.Brakus656@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "363 HIGHLAND AVENUE" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02720", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788", - "resource": { - "resourceType": "Encounter", - "id": "93bc0473-3b31-4a1e-8456-382950f8b788", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "individual": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - } - ], - "period": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe", - "resource": { - "resourceType": "Condition", - "id": "fabcf585-727e-4e48-bcac-2f51c1588ebe", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - }, - "onsetDateTime": "2012-05-15T09:41:37-07:00", - "abatementDateTime": "2012-06-19T09:41:37-07:00", - "recordedDate": "2012-05-15T09:41:37-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:1123b665-2688-4488-bd77-96f11c96ba58", - "resource": { - "resourceType": "MedicationRequest", - "id": "1123b665-2688-4488-bd77-96f11c96ba58", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - }, - "authoredOn": "2012-05-15T09:41:37-07:00", - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:457d2594-ad89-4c5a-a863-eb983d000719", - "resource": { - "resourceType": "Claim", - "id": "457d2594-ad89-4c5a-a863-eb983d000719", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "created": "2012-05-15T10:41:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:1123b665-2688-4488-bd77-96f11c96ba58" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - } - ] - } - ], - "total": { - "value": 4.14, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba", - "resource": { - "resourceType": "CareTeam", - "id": "d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba", - "status": "inactive", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - }, - "period": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-06-19T09:41:37-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:50175367-bf28-4bc0-87d6-88f8dc0f984e", - "resource": { - "resourceType": "CarePlan", - "id": "50175367-bf28-4bc0-87d6-88f8dc0f984e", - "text": { - "status": "generated", - "div": "
    Care Plan for Physical therapy procedure.
    Activities:
    • Physical therapy procedure
    • Physical therapy procedure

    Care plan is meant to treat Sprain of ankle.
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "91251008", - "display": "Physical therapy procedure" - } - ], - "text": "Physical therapy procedure" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - }, - "period": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-06-19T09:41:37-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:d35fa5f3-6cd5-44ec-87e8-ac46f936c7ba" - } - ], - "addresses": [ - { - "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229586001", - "display": "Rest, ice, compression and elevation treatment programme" - } - ], - "text": "Rest, ice, compression and elevation treatment programme" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229070002", - "display": "Stretching exercises" - } - ], - "text": "Stretching exercises" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:f285810f-2b70-4abb-afdd-db15790f17f1", - "resource": { - "resourceType": "Claim", - "id": "f285810f-2b70-4abb-afdd-db15790f17f1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "created": "2012-05-15T10:41:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1c10ff57-e9fa-4c09-838c-05b6854df0f8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1c10ff57-e9fa-4c09-838c-05b6854df0f8", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "performer": [ - { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f285810f-2b70-4abb-afdd-db15790f17f1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-05-15T10:41:37-07:00", - "end": "2013-05-15T10:41:37-07:00" - }, - "created": "2012-05-15T10:41:37-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:f285810f-2b70-4abb-afdd-db15790f17f1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "servicedPeriod": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:93bc0473-3b31-4a1e-8456-382950f8b788" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - }, - "servicedPeriod": { - "start": "2012-05-15T09:41:37-07:00", - "end": "2012-05-15T10:41:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff", - "resource": { - "resourceType": "Encounter", - "id": "57add463-aac6-4cdb-9556-6e552d8c67ff", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "individual": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - } - ], - "period": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24", - "resource": { - "resourceType": "Condition", - "id": "f3f25715-3afa-4181-848b-c1b9c3ccaf24", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - }, - "onsetDateTime": "2012-07-02T09:41:37-07:00", - "abatementDateTime": "2012-07-09T09:41:37-07:00", - "recordedDate": "2012-07-02T09:41:37-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:a1a731b3-96ee-4935-b989-4f2db5f32a12", - "resource": { - "resourceType": "Procedure", - "id": "a1a731b3-96ee-4935-b989-4f2db5f32a12", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - }, - "performedPeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:02:37-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24", - "display": "Acute bronchitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d59314ed-ffb4-4bb9-8c31-37b46e7f6c39", - "resource": { - "resourceType": "MedicationRequest", - "id": "d59314ed-ffb4-4bb9-8c31-37b46e7f6c39", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1043400", - "display": "Acetaminophen 21.7 MG/ML / Dextromethorphan Hydrobromide 1 MG/ML / doxylamine succinate 0.417 MG/ML Oral Solution" - } - ], - "text": "Acetaminophen 21.7 MG/ML / Dextromethorphan Hydrobromide 1 MG/ML / doxylamine succinate 0.417 MG/ML Oral Solution" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - }, - "authoredOn": "2012-07-02T09:41:37-07:00", - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - }, - "reasonReference": [ - { - "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:05a6d60b-dadd-40c9-8505-17d16918446f", - "resource": { - "resourceType": "Claim", - "id": "05a6d60b-dadd-40c9-8505-17d16918446f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "created": "2012-07-02T10:17:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:d59314ed-ffb4-4bb9-8c31-37b46e7f6c39" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - } - ] - } - ], - "total": { - "value": 10.07, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9b2c09bd-bdda-4037-b00d-97c5faa0d735", - "resource": { - "resourceType": "CareTeam", - "id": "9b2c09bd-bdda-4037-b00d-97c5faa0d735", - "status": "inactive", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - }, - "period": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-12-31T08:41:37-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:433e5bff-790b-4f36-8ffd-f64cde69dc58", - "resource": { - "resourceType": "CarePlan", - "id": "433e5bff-790b-4f36-8ffd-f64cde69dc58", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - }, - "period": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-12-31T08:41:37-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:9b2c09bd-bdda-4037-b00d-97c5faa0d735" - } - ], - "addresses": [ - { - "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:0dba802b-36c4-4736-b27d-2e61ed03dd50", - "resource": { - "resourceType": "Claim", - "id": "0dba802b-36c4-4736-b27d-2e61ed03dd50", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "created": "2012-07-02T10:17:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a1a731b3-96ee-4935-b989-4f2db5f32a12" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8a2c0a9c-1e72-4839-96e1-e81ec44d5ed1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8a2c0a9c-1e72-4839-96e1-e81ec44d5ed1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "performer": [ - { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0dba802b-36c4-4736-b27d-2e61ed03dd50" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-07-02T10:17:37-07:00", - "end": "2013-07-02T10:17:37-07:00" - }, - "created": "2012-07-02T10:17:37-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0dba802b-36c4-4736-b27d-2e61ed03dd50" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f3f25715-3afa-4181-848b-c1b9c3ccaf24" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:57add463-aac6-4cdb-9556-6e552d8c67ff" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "servicedPeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "servicedPeriod": { - "start": "2012-07-02T09:41:37-07:00", - "end": "2012-07-02T10:17:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "resource": { - "resourceType": "Organization", - "id": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "TRU MEDICAL WALK IN CENTER", - "telecom": [ - { - "system": "phone", - "value": "508-675-1522" - } - ], - "address": [ - { - "line": [ - "528 NEWTON STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "2721", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", - "resource": { - "resourceType": "Practitioner", - "id": "21a54030-cce5-3104-a773-9eb9bd60830b", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906799" - } - ], - "active": true, - "name": [ - { - "family": "Eichmann909", - "given": [ - "Wanetta360" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Wanetta360.Eichmann909@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "528 NEWTON STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "2721", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef", - "resource": { - "resourceType": "Encounter", - "id": "bd8588cc-ada9-4ac4-bcb0-49868d1c30ef", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-07-09T09:41:37-07:00", - "end": "2012-07-09T09:56:37-07:00" - }, - "individual": { - "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", - "display": "Dr. Wanetta360 Eichmann909" - } - } - ], - "period": { - "start": "2012-07-09T09:41:37-07:00", - "end": "2012-07-09T09:56:37-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "display": "TRU MEDICAL WALK IN CENTER" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b9ce49ce-a850-4fdc-8672-e0b222c42f94", - "resource": { - "resourceType": "Immunization", - "id": "b9ce49ce-a850-4fdc-8672-e0b222c42f94", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" - }, - "occurrenceDateTime": "2012-07-09T09:41:37-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b66781db-8de6-441f-ada7-a1304cf4cb0f", - "resource": { - "resourceType": "Claim", - "id": "b66781db-8de6-441f-ada7-a1304cf4cb0f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2012-07-09T09:41:37-07:00", - "end": "2012-07-09T09:56:37-07:00" - }, - "created": "2012-07-09T09:56:37-07:00", - "provider": { - "reference": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "display": "TRU MEDICAL WALK IN CENTER" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:b9ce49ce-a850-4fdc-8672-e0b222c42f94" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:de603239-8f97-4d7d-ba66-a35f1c4f99b0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "de603239-8f97-4d7d-ba66-a35f1c4f99b0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" - }, - "performer": [ - { - "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b66781db-8de6-441f-ada7-a1304cf4cb0f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-07-09T09:56:37-07:00", - "end": "2013-07-09T09:56:37-07:00" - }, - "created": "2012-07-09T09:56:37-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b66781db-8de6-441f-ada7-a1304cf4cb0f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - }, - "servicedPeriod": { - "start": "2012-07-09T09:41:37-07:00", - "end": "2012-07-09T09:56:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:bd8588cc-ada9-4ac4-bcb0-49868d1c30ef" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2012-07-09T09:41:37-07:00", - "end": "2012-07-09T09:56:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71", - "resource": { - "resourceType": "Encounter", - "id": "d44f687b-ec6d-400f-b784-1c61498adb71", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T09:11:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T09:11:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7c58963d-ef76-4731-ac96-008f443cdb97", - "resource": { - "resourceType": "Observation", - "id": "7c58963d-ef76-4731-ac96-008f443cdb97", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 131, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:48c40b57-7b57-4672-ac50-2abe158c0d71", - "resource": { - "resourceType": "Observation", - "id": "48c40b57-7b57-4672-ac50-2abe158c0d71", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db2345b6-c9d2-4774-921d-e9fbe6263d76", - "resource": { - "resourceType": "Observation", - "id": "db2345b6-c9d2-4774-921d-e9fbe6263d76", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 35.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8cd18bd4-4b06-4210-a5e1-379c4765e7b1", - "resource": { - "resourceType": "Observation", - "id": "8cd18bd4-4b06-4210-a5e1-379c4765e7b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 20.57, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a31a8e8-266e-45e8-a4f2-8a7d54686f6c", - "resource": { - "resourceType": "Observation", - "id": "1a31a8e8-266e-45e8-a4f2-8a7d54686f6c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 93.83, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8cda67d1-1480-42be-8aff-f0ea4268af9e", - "resource": { - "resourceType": "Observation", - "id": "8cda67d1-1480-42be-8aff-f0ea4268af9e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 131, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6f957375-798f-4922-a544-655d460d9a4d", - "resource": { - "resourceType": "Observation", - "id": "6f957375-798f-4922-a544-655d460d9a4d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 83, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:696fd53f-9007-4d8b-8b37-7092221d19c6", - "resource": { - "resourceType": "Observation", - "id": "696fd53f-9007-4d8b-8b37-7092221d19c6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:483b6520-0ec7-4b8d-bd4a-8622edd7cb4c", - "resource": { - "resourceType": "Observation", - "id": "483b6520-0ec7-4b8d-bd4a-8622edd7cb4c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "effectiveDateTime": "2012-12-31T08:41:37-08:00", - "issued": "2012-12-31T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36dc9d54-0462-4f5d-adfb-77a3cbeeebf9", - "resource": { - "resourceType": "Procedure", - "id": "36dc9d54-0462-4f5d-adfb-77a3cbeeebf9", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - }, - "performedPeriod": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T08:56:37-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3d1ef59c-db3f-417b-9c38-8d49f01fd2f2", - "resource": { - "resourceType": "Claim", - "id": "3d1ef59c-db3f-417b-9c38-8d49f01fd2f2", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T09:11:37-08:00" - }, - "created": "2012-12-31T09:11:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:36dc9d54-0462-4f5d-adfb-77a3cbeeebf9" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 799.48, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b020878c-ebb4-4192-a5f7-5fdb85be74fd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b020878c-ebb4-4192-a5f7-5fdb85be74fd", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3d1ef59c-db3f-417b-9c38-8d49f01fd2f2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2012-12-31T09:11:37-08:00", - "end": "2013-12-31T09:11:37-08:00" - }, - "created": "2012-12-31T09:11:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3d1ef59c-db3f-417b-9c38-8d49f01fd2f2" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d44f687b-ec6d-400f-b784-1c61498adb71" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2012-12-31T08:41:37-08:00", - "end": "2012-12-31T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 799.48, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 159.89600000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 639.5840000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 799.48, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 799.48, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 639.5840000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206", - "resource": { - "resourceType": "Encounter", - "id": "531fa623-ec6e-4e73-bff2-65ea7160f206", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-01-06T08:41:37-08:00", - "end": "2014-01-06T08:56:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2014-01-06T08:41:37-08:00", - "end": "2014-01-06T08:56:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:911f9129-1d1b-4c1f-9acc-15c27f191185", - "resource": { - "resourceType": "Observation", - "id": "911f9129-1d1b-4c1f-9acc-15c27f191185", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 135.9, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b2cdc4f-238f-49d8-aa58-211f5a9b78b3", - "resource": { - "resourceType": "Observation", - "id": "8b2cdc4f-238f-49d8-aa58-211f5a9b78b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bea7dae0-b453-4e64-8ab7-456037ff47c2", - "resource": { - "resourceType": "Observation", - "id": "bea7dae0-b453-4e64-8ab7-456037ff47c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 40.1, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9afd6319-2f22-4d1e-8554-b5d7d7d6a95a", - "resource": { - "resourceType": "Observation", - "id": "9afd6319-2f22-4d1e-8554-b5d7d7d6a95a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 21.72, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1c4ffe7-2f05-420f-ab2a-fbe5e974075f", - "resource": { - "resourceType": "Observation", - "id": "d1c4ffe7-2f05-420f-ab2a-fbe5e974075f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 94.162, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b08c7782-3d0d-4cba-aa50-5317d1e8470e", - "resource": { - "resourceType": "Observation", - "id": "b08c7782-3d0d-4cba-aa50-5317d1e8470e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 122, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2adb630-7077-451d-ae92-8e58274bf51b", - "resource": { - "resourceType": "Observation", - "id": "a2adb630-7077-451d-ae92-8e58274bf51b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 81, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca9ac15b-0d6d-4882-91b9-7b829398401e", - "resource": { - "resourceType": "Observation", - "id": "ca9ac15b-0d6d-4882-91b9-7b829398401e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e828e94-03a0-41aa-80fb-a265a296fe1d", - "resource": { - "resourceType": "Observation", - "id": "4e828e94-03a0-41aa-80fb-a265a296fe1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "effectiveDateTime": "2014-01-06T08:41:37-08:00", - "issued": "2014-01-06T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cad79faf-ed70-4afb-864b-c3b20c2a5c06", - "resource": { - "resourceType": "Immunization", - "id": "cad79faf-ed70-4afb-864b-c3b20c2a5c06", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - }, - "occurrenceDateTime": "2014-01-06T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e76109aa-f769-4340-ae67-6f9edbb24577", - "resource": { - "resourceType": "Claim", - "id": "e76109aa-f769-4340-ae67-6f9edbb24577", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2014-01-06T08:41:37-08:00", - "end": "2014-01-06T08:56:37-08:00" - }, - "created": "2014-01-06T08:56:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:cad79faf-ed70-4afb-864b-c3b20c2a5c06" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:063d2c05-1305-48d6-8a94-06a3355d164a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "063d2c05-1305-48d6-8a94-06a3355d164a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e76109aa-f769-4340-ae67-6f9edbb24577" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2014-01-06T08:56:37-08:00", - "end": "2015-01-06T08:56:37-08:00" - }, - "created": "2014-01-06T08:56:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e76109aa-f769-4340-ae67-6f9edbb24577" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2014-01-06T08:41:37-08:00", - "end": "2014-01-06T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:531fa623-ec6e-4e73-bff2-65ea7160f206" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2014-01-06T08:41:37-08:00", - "end": "2014-01-06T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6", - "resource": { - "resourceType": "Encounter", - "id": "55af649f-e26b-4710-b743-391f565d92f6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:762fe22e-568b-4838-b52b-2f3afec349b1", - "resource": { - "resourceType": "Observation", - "id": "762fe22e-568b-4838-b52b-2f3afec349b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 140.7, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e8cf17f-c9e2-4c54-ab73-b3f42a7b8c6b", - "resource": { - "resourceType": "Observation", - "id": "3e8cf17f-c9e2-4c54-ab73-b3f42a7b8c6b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6d79712-8bf9-4e82-83f8-c1b01ef824da", - "resource": { - "resourceType": "Observation", - "id": "c6d79712-8bf9-4e82-83f8-c1b01ef824da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 45.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0a275634-9c7c-4fa9-a5c0-4f6617b00f8c", - "resource": { - "resourceType": "Observation", - "id": "0a275634-9c7c-4fa9-a5c0-4f6617b00f8c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 22.92, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25a56f1c-b3e6-40a3-8206-6d31ed6ba574", - "resource": { - "resourceType": "Observation", - "id": "25a56f1c-b3e6-40a3-8206-6d31ed6ba574", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 94.498, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4da129ee-3bc5-436a-9948-6cab80d8ec1a", - "resource": { - "resourceType": "Observation", - "id": "4da129ee-3bc5-436a-9948-6cab80d8ec1a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 128, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:06957817-fdcc-421c-b791-44812931b932", - "resource": { - "resourceType": "Observation", - "id": "06957817-fdcc-421c-b791-44812931b932", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df847d91-3de2-40bb-9346-8654369d6ada", - "resource": { - "resourceType": "Observation", - "id": "df847d91-3de2-40bb-9346-8654369d6ada", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2a7e52a9-d189-43e1-9873-d8431e04a3d0", - "resource": { - "resourceType": "Observation", - "id": "2a7e52a9-d189-43e1-9873-d8431e04a3d0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 8.1514, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", - "resource": { - "resourceType": "Observation", - "id": "1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 4.5262, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6b92048-7c77-4773-b826-3639d7fcdf7c", - "resource": { - "resourceType": "Observation", - "id": "e6b92048-7c77-4773-b826-3639d7fcdf7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 13.159, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:948a0894-e573-4073-8c7b-97a26ba9edc2", - "resource": { - "resourceType": "Observation", - "id": "948a0894-e573-4073-8c7b-97a26ba9edc2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 35.212, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9b050885-8ea7-40d9-a9fa-02504f352846", - "resource": { - "resourceType": "Observation", - "id": "9b050885-8ea7-40d9-a9fa-02504f352846", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 92.463, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ea022d5-dd3f-436a-8d2d-2130e65d616e", - "resource": { - "resourceType": "Observation", - "id": "2ea022d5-dd3f-436a-8d2d-2130e65d616e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 27.982, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5aac06c7-9de2-4ff2-97cd-d1551ac83457", - "resource": { - "resourceType": "Observation", - "id": "5aac06c7-9de2-4ff2-97cd-d1551ac83457", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 34.283, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51622cb4-afab-4d28-993e-49286e3e9762", - "resource": { - "resourceType": "Observation", - "id": "51622cb4-afab-4d28-993e-49286e3e9762", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 39.384, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32e37758-5aa6-404b-bfa2-7e121d3599af", - "resource": { - "resourceType": "Observation", - "id": "32e37758-5aa6-404b-bfa2-7e121d3599af", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 358.98, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", - "resource": { - "resourceType": "Observation", - "id": "8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 295.93, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:426478df-b366-4f7c-ac2e-05bde99fe385", - "resource": { - "resourceType": "Observation", - "id": "426478df-b366-4f7c-ac2e-05bde99fe385", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueQuantity": { - "value": 9.4896, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e0cd77c-31e3-42a3-af98-36897c387b1f", - "resource": { - "resourceType": "Observation", - "id": "5e0cd77c-31e3-42a3-af98-36897c387b1f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7892a1fd-0d95-45e4-bea6-cc613b4a6e0d", - "resource": { - "resourceType": "Immunization", - "id": "7892a1fd-0d95-45e4-bea6-cc613b4a6e0d", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "115", - "display": "Tdap" - } - ], - "text": "Tdap" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "occurrenceDateTime": "2015-01-12T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8a6f3520-1ef7-4370-b54e-c6d87ca69fd4", - "resource": { - "resourceType": "Immunization", - "id": "8a6f3520-1ef7-4370-b54e-c6d87ca69fd4", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "occurrenceDateTime": "2015-01-12T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:eff3f274-7291-4850-9944-2247b963fd5c", - "resource": { - "resourceType": "Immunization", - "id": "eff3f274-7291-4850-9944-2247b963fd5c", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "occurrenceDateTime": "2015-01-12T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:98589c22-9e19-431f-a2fe-7156553c4d7e", - "resource": { - "resourceType": "Immunization", - "id": "98589c22-9e19-431f-a2fe-7156553c4d7e", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "occurrenceDateTime": "2015-01-12T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c5944ac4-b35d-44b7-b7a4-1a8eeec0c2d8", - "resource": { - "resourceType": "DiagnosticReport", - "id": "c5944ac4-b35d-44b7-b7a4-1a8eeec0c2d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - }, - "effectiveDateTime": "2015-01-12T08:41:37-08:00", - "issued": "2015-01-12T08:41:37.844-08:00", - "result": [ - { - "reference": "urn:uuid:2a7e52a9-d189-43e1-9873-d8431e04a3d0", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1a57baed-7a66-4f2a-8fec-e3c1cbf064a6", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:e6b92048-7c77-4773-b826-3639d7fcdf7c", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:948a0894-e573-4073-8c7b-97a26ba9edc2", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:9b050885-8ea7-40d9-a9fa-02504f352846", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:2ea022d5-dd3f-436a-8d2d-2130e65d616e", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:5aac06c7-9de2-4ff2-97cd-d1551ac83457", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:51622cb4-afab-4d28-993e-49286e3e9762", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:32e37758-5aa6-404b-bfa2-7e121d3599af", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:8aba8a49-7c84-49aa-b0b9-3c1a34783f4b", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:426478df-b366-4f7c-ac2e-05bde99fe385", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:4fbfb167-03e9-46b4-890a-39d436328462", - "resource": { - "resourceType": "Claim", - "id": "4fbfb167-03e9-46b4-890a-39d436328462", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "created": "2015-01-12T08:56:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7892a1fd-0d95-45e4-bea6-cc613b4a6e0d" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8a6f3520-1ef7-4370-b54e-c6d87ca69fd4" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:eff3f274-7291-4850-9944-2247b963fd5c" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:98589c22-9e19-431f-a2fe-7156553c4d7e" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "115", - "display": "Tdap" - } - ], - "text": "Tdap" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 5, - "informationSequence": [ - 4 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f401b610-f7fb-4a70-b4ab-7f2a3ce950b8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f401b610-f7fb-4a70-b4ab-7f2a3ce950b8", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4fbfb167-03e9-46b4-890a-39d436328462" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2015-01-12T08:56:37-08:00", - "end": "2016-01-12T08:56:37-08:00" - }, - "created": "2015-01-12T08:56:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4fbfb167-03e9-46b4-890a-39d436328462" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:55af649f-e26b-4710-b743-391f565d92f6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "115", - "display": "Tdap" - } - ], - "text": "Tdap" - }, - "servicedPeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "servicedPeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "informationSequence": [ - 4 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "servicedPeriod": { - "start": "2015-01-12T08:41:37-08:00", - "end": "2015-01-12T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 449.66400000000004, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a", - "resource": { - "resourceType": "Encounter", - "id": "0b75bac0-fd68-44e7-8d14-9a6b8331ac2a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:da815f14-4818-4053-982a-50b3be7b6460", - "resource": { - "resourceType": "Observation", - "id": "da815f14-4818-4053-982a-50b3be7b6460", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 146, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8bae7e83-543a-4dea-8ad0-d53cab2009a4", - "resource": { - "resourceType": "Observation", - "id": "8bae7e83-543a-4dea-8ad0-d53cab2009a4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5fe9f9de-55ba-499a-8c16-e292600e153c", - "resource": { - "resourceType": "Observation", - "id": "5fe9f9de-55ba-499a-8c16-e292600e153c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 51.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49187e2c-56fc-44d8-98f9-41c94ca73632", - "resource": { - "resourceType": "Observation", - "id": "49187e2c-56fc-44d8-98f9-41c94ca73632", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 24.15, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d5f434bb-71ca-4b4f-aebe-dbb1ee4fda50", - "resource": { - "resourceType": "Observation", - "id": "d5f434bb-71ca-4b4f-aebe-dbb1ee4fda50", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 94.877, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ef2f58d6-526d-44c0-91ab-0234b58caad3", - "resource": { - "resourceType": "Observation", - "id": "ef2f58d6-526d-44c0-91ab-0234b58caad3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 120, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c7fb9fb1-6f9c-4842-8d4e-663e9b9991a0", - "resource": { - "resourceType": "Observation", - "id": "c7fb9fb1-6f9c-4842-8d4e-663e9b9991a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 69, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c1a567a-bc88-4bae-b4a5-0b4fb7ecbd15", - "resource": { - "resourceType": "Observation", - "id": "9c1a567a-bc88-4bae-b4a5-0b4fb7ecbd15", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:09edde80-b6ed-4d39-8bc9-47eeb6786ead", - "resource": { - "resourceType": "Observation", - "id": "09edde80-b6ed-4d39-8bc9-47eeb6786ead", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "effectiveDateTime": "2016-01-18T08:41:37-08:00", - "issued": "2016-01-18T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1854baa9-8629-49e5-a0f1-d03029099fa0", - "resource": { - "resourceType": "Immunization", - "id": "1854baa9-8629-49e5-a0f1-d03029099fa0", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "occurrenceDateTime": "2016-01-18T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:64fce6b8-8e84-4be2-8878-90e0b1716d26", - "resource": { - "resourceType": "Immunization", - "id": "64fce6b8-8e84-4be2-8878-90e0b1716d26", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - }, - "occurrenceDateTime": "2016-01-18T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7566ba8c-7330-42dd-94f6-2672e95e33e4", - "resource": { - "resourceType": "Claim", - "id": "7566ba8c-7330-42dd-94f6-2672e95e33e4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "created": "2016-01-18T08:56:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1854baa9-8629-49e5-a0f1-d03029099fa0" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:64fce6b8-8e84-4be2-8878-90e0b1716d26" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7d43aefc-7587-4b2e-9c73-572ec00d2c9b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7d43aefc-7587-4b2e-9c73-572ec00d2c9b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7566ba8c-7330-42dd-94f6-2672e95e33e4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2016-01-18T08:56:37-08:00", - "end": "2017-01-18T08:56:37-08:00" - }, - "created": "2016-01-18T08:56:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7566ba8c-7330-42dd-94f6-2672e95e33e4" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0b75bac0-fd68-44e7-8d14-9a6b8331ac2a" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "servicedPeriod": { - "start": "2016-01-18T08:41:37-08:00", - "end": "2016-01-18T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 224.83200000000002, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99", - "resource": { - "resourceType": "Encounter", - "id": "b4c7f893-7782-42e5-9b39-6967e2e2fb99", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "individual": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - } - ], - "period": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e", - "resource": { - "resourceType": "Condition", - "id": "bc332dc7-b8c1-4a1f-9094-12b95e19ed3e", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - }, - "onsetDateTime": "2016-08-27T09:41:37-07:00", - "abatementDateTime": "2016-09-10T09:41:37-07:00", - "recordedDate": "2016-08-27T09:41:37-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:40ccbe9e-36a5-4c01-ab7a-e743c0a74766", - "resource": { - "resourceType": "MedicationRequest", - "id": "40ccbe9e-36a5-4c01-ab7a-e743c0a74766", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - }, - "authoredOn": "2016-08-27T09:41:37-07:00", - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:6db7fa8c-4dc3-4e06-8135-76d4a101871c", - "resource": { - "resourceType": "Claim", - "id": "6db7fa8c-4dc3-4e06-8135-76d4a101871c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "created": "2016-08-27T10:41:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:40ccbe9e-36a5-4c01-ab7a-e743c0a74766" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - } - ] - } - ], - "total": { - "value": 6.72, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f", - "resource": { - "resourceType": "CareTeam", - "id": "ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f", - "status": "inactive", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - }, - "period": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-09-10T09:41:37-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:a135cdd9-af11-4552-9f35-9a6dd6d952a9", - "resource": { - "resourceType": "CarePlan", - "id": "a135cdd9-af11-4552-9f35-9a6dd6d952a9", - "text": { - "status": "generated", - "div": "
    Care Plan for Physical therapy procedure.
    Activities:
    • Physical therapy procedure
    • Physical therapy procedure

    Care plan is meant to treat Sprain of ankle.
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "91251008", - "display": "Physical therapy procedure" - } - ], - "text": "Physical therapy procedure" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - }, - "period": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-09-10T09:41:37-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:ff51b76e-bf7d-4512-bc5b-1a1c521e9d1f" - } - ], - "addresses": [ - { - "reference": "urn:uuid:fabcf585-727e-4e48-bcac-2f51c1588ebe" - }, - { - "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229586001", - "display": "Rest, ice, compression and elevation treatment programme" - } - ], - "text": "Rest, ice, compression and elevation treatment programme" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229070002", - "display": "Stretching exercises" - } - ], - "text": "Stretching exercises" - }, - "status": "completed", - "location": { - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:e102d5db-4bf7-4fbe-8dc7-315a366ac4cb", - "resource": { - "resourceType": "Claim", - "id": "e102d5db-4bf7-4fbe-8dc7-315a366ac4cb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "created": "2016-08-27T10:41:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f28cfb37-5d1f-4d44-9176-65343a68862f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f28cfb37-5d1f-4d44-9176-65343a68862f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "performer": [ - { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e102d5db-4bf7-4fbe-8dc7-315a366ac4cb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2016-08-27T10:41:37-07:00", - "end": "2017-08-27T10:41:37-07:00" - }, - "created": "2016-08-27T10:41:37-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e102d5db-4bf7-4fbe-8dc7-315a366ac4cb" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:bc332dc7-b8c1-4a1f-9094-12b95e19ed3e" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "servicedPeriod": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b4c7f893-7782-42e5-9b39-6967e2e2fb99" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - }, - "servicedPeriod": { - "start": "2016-08-27T09:41:37-07:00", - "end": "2016-08-27T10:41:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758", - "resource": { - "resourceType": "Encounter", - "id": "a08c5912-933e-4da3-8100-31264995b758", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:60bf9346-3ae2-45a0-8149-3cdf03bbe7d7", - "resource": { - "resourceType": "Observation", - "id": "60bf9346-3ae2-45a0-8149-3cdf03bbe7d7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 153, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d410a58a-3c95-4007-ab5a-6da844a168ef", - "resource": { - "resourceType": "Observation", - "id": "d410a58a-3c95-4007-ab5a-6da844a168ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d890600-5da7-4eaa-8bca-e5da92d11a4e", - "resource": { - "resourceType": "Observation", - "id": "1d890600-5da7-4eaa-8bca-e5da92d11a4e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 64.9, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69acd0f8-b26c-4bf3-bc27-c253cf448694", - "resource": { - "resourceType": "Observation", - "id": "69acd0f8-b26c-4bf3-bc27-c253cf448694", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 27.73, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:039a00ad-eb92-490a-83da-4b71ed0a2698", - "resource": { - "resourceType": "Observation", - "id": "039a00ad-eb92-490a-83da-4b71ed0a2698", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 97.488, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f2f9a723-0f49-4672-b479-ced8a850cf19", - "resource": { - "resourceType": "Observation", - "id": "f2f9a723-0f49-4672-b479-ced8a850cf19", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 115, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69a22dae-c5c7-4da0-8c61-bb3944504c31", - "resource": { - "resourceType": "Observation", - "id": "69a22dae-c5c7-4da0-8c61-bb3944504c31", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 64, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6068bc02-26f0-432f-9ff8-c4bcdbc414f2", - "resource": { - "resourceType": "Observation", - "id": "6068bc02-26f0-432f-9ff8-c4bcdbc414f2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9fa3128-43e2-4ea6-9900-0c73c826d4bc", - "resource": { - "resourceType": "Observation", - "id": "c9fa3128-43e2-4ea6-9900-0c73c826d4bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "effectiveDateTime": "2017-01-23T08:41:37-08:00", - "issued": "2017-01-23T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e965c8e-bcc9-4c3a-be01-ecafbf4f6094", - "resource": { - "resourceType": "Procedure", - "id": "0e965c8e-bcc9-4c3a-be01-ecafbf4f6094", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "performedPeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T08:56:37-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:95c09b71-8c04-4778-922a-c5d207e7c989", - "resource": { - "resourceType": "Immunization", - "id": "95c09b71-8c04-4778-922a-c5d207e7c989", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "occurrenceDateTime": "2017-01-23T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e3b92940-d554-47b0-ac36-62bcec08f3dc", - "resource": { - "resourceType": "Immunization", - "id": "e3b92940-d554-47b0-ac36-62bcec08f3dc", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - }, - "occurrenceDateTime": "2017-01-23T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a488e6d7-7b1a-4383-a5b2-fc61327f9fd0", - "resource": { - "resourceType": "Claim", - "id": "a488e6d7-7b1a-4383-a5b2-fc61327f9fd0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "created": "2017-01-23T09:11:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:95c09b71-8c04-4778-922a-c5d207e7c989" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e3b92940-d554-47b0-ac36-62bcec08f3dc" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:0e965c8e-bcc9-4c3a-be01-ecafbf4f6094" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 577.47, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:410f2644-845c-4d89-9dbd-b712c4bf8f3b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "410f2644-845c-4d89-9dbd-b712c4bf8f3b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a488e6d7-7b1a-4383-a5b2-fc61327f9fd0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2017-01-23T09:11:37-08:00", - "end": "2018-01-23T09:11:37-08:00" - }, - "created": "2017-01-23T09:11:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a488e6d7-7b1a-4383-a5b2-fc61327f9fd0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a08c5912-933e-4da3-8100-31264995b758" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "servicedPeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T08:41:37-08:00", - "end": "2017-01-23T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 577.47, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 115.49400000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 461.97600000000006, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 577.47, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 577.47, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 686.8080000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695", - "resource": { - "resourceType": "Encounter", - "id": "2f7612c4-f9c3-4343-90c5-ae3e8e9c9695", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a", - "resource": { - "resourceType": "Condition", - "id": "9b1f80d9-3e7a-49ba-9371-102c0af0803a", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "onsetDateTime": "2018-01-29T08:41:37-08:00", - "recordedDate": "2018-01-29T08:41:37-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:054fc06a-e893-4090-8688-1b400253524f", - "resource": { - "resourceType": "Observation", - "id": "054fc06a-e893-4090-8688-1b400253524f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 161.2, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d78804de-9030-4fe2-8173-9c729ab82186", - "resource": { - "resourceType": "Observation", - "id": "d78804de-9030-4fe2-8173-9c729ab82186", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d57a613-7f09-4a56-9d45-3476a8a6edc1", - "resource": { - "resourceType": "Observation", - "id": "5d57a613-7f09-4a56-9d45-3476a8a6edc1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 79.2, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:093bb0b2-cfc0-4e0b-9b4c-5a1a4eddddc3", - "resource": { - "resourceType": "Observation", - "id": "093bb0b2-cfc0-4e0b-9b4c-5a1a4eddddc3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 30.49, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:398ea67e-6187-4760-8e99-c757d5d4148a", - "resource": { - "resourceType": "Observation", - "id": "398ea67e-6187-4760-8e99-c757d5d4148a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 98.364, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4245b809-6791-4fc6-aea1-55a0989fd59e", - "resource": { - "resourceType": "Observation", - "id": "4245b809-6791-4fc6-aea1-55a0989fd59e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 115, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7104eb83-3044-4b45-ad7b-67cd4891f1ef", - "resource": { - "resourceType": "Observation", - "id": "7104eb83-3044-4b45-ad7b-67cd4891f1ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b76f04f1-aa2f-4d0b-aea0-8b08012dd09f", - "resource": { - "resourceType": "Observation", - "id": "b76f04f1-aa2f-4d0b-aea0-8b08012dd09f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:58510814-1edf-44a6-8fbf-43171f1a0f30", - "resource": { - "resourceType": "Observation", - "id": "58510814-1edf-44a6-8fbf-43171f1a0f30", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "effectiveDateTime": "2018-01-29T08:41:37-08:00", - "issued": "2018-01-29T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e755db2e-b8a7-4b64-8fa3-ebf09625571c", - "resource": { - "resourceType": "Immunization", - "id": "e755db2e-b8a7-4b64-8fa3-ebf09625571c", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - }, - "occurrenceDateTime": "2018-01-29T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:feed7acd-8ff0-42cc-b0da-725a1b513d90", - "resource": { - "resourceType": "Claim", - "id": "feed7acd-8ff0-42cc-b0da-725a1b513d90", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "created": "2018-01-29T08:56:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e755db2e-b8a7-4b64-8fa3-ebf09625571c" - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e55a3e5c-8524-4a1a-93bc-5c13b3b71b74", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e55a3e5c-8524-4a1a-93bc-5c13b3b71b74", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "feed7acd-8ff0-42cc-b0da-725a1b513d90" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2018-01-29T08:56:37-08:00", - "end": "2019-01-29T08:56:37-08:00" - }, - "created": "2018-01-29T08:56:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:feed7acd-8ff0-42cc-b0da-725a1b513d90" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9b1f80d9-3e7a-49ba-9371-102c0af0803a" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2f7612c4-f9c3-4343-90c5-ae3e8e9c9695" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "servicedPeriod": { - "start": "2018-01-29T08:41:37-08:00", - "end": "2018-01-29T08:56:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1", - "resource": { - "resourceType": "Encounter", - "id": "e1157598-1cdd-4242-812d-a3ab228a69e1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "individual": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - } - } - ], - "period": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f", - "resource": { - "resourceType": "Condition", - "id": "b913dddb-6e54-452e-9559-7fdfd7356a1f", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - }, - "onsetDateTime": "2018-10-05T09:41:37-07:00", - "abatementDateTime": "2018-10-14T09:41:37-07:00", - "recordedDate": "2018-10-05T09:41:37-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b830c0e1-09fe-4ea9-b55c-fef6cebaa94b", - "resource": { - "resourceType": "Observation", - "id": "b830c0e1-09fe-4ea9-b55c-fef6cebaa94b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - }, - { - "system": "http://loinc.org", - "code": "8331-1", - "display": "Oral temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - }, - "effectiveDateTime": "2018-10-05T09:41:37-07:00", - "issued": "2018-10-05T09:41:37.844-07:00", - "valueQuantity": { - "value": 39.25, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70", - "resource": { - "resourceType": "MedicationRequest", - "id": "7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "834061", - "display": "Penicillin V Potassium 250 MG Oral Tablet" - } - ], - "text": "Penicillin V Potassium 250 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - }, - "authoredOn": "2018-10-05T09:41:37-07:00", - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "display": "Dr. Gillian484 Brakus656" - }, - "reasonReference": [ - { - "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1aef6704-cada-4b33-b7e3-b5f6c16389a1", - "resource": { - "resourceType": "Claim", - "id": "1aef6704-cada-4b33-b7e3-b5f6c16389a1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "created": "2018-10-05T09:56:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:7df0f8b1-0ae0-4b06-a6e5-852a8a3c8c70" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - } - ] - } - ], - "total": { - "value": 18.31, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7cabe084-1ba9-43cc-bddb-3e34b858abf7", - "resource": { - "resourceType": "Claim", - "id": "7cabe084-1ba9-43cc-bddb-3e34b858abf7", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "created": "2018-10-05T09:56:37-07:00", - "provider": { - "reference": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "display": "SOUTHCOAST HOSPITAL GROUP, INC" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:71dc2f89-9a20-4f31-85e4-ef582fc534aa", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "71dc2f89-9a20-4f31-85e4-ef582fc534aa", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "performer": [ - { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7cabe084-1ba9-43cc-bddb-3e34b858abf7" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2018-10-05T09:56:37-07:00", - "end": "2019-10-05T09:56:37-07:00" - }, - "created": "2018-10-05T09:56:37-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7cabe084-1ba9-43cc-bddb-3e34b858abf7" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b913dddb-6e54-452e-9559-7fdfd7356a1f" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e1157598-1cdd-4242-812d-a3ab228a69e1" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - }, - "servicedPeriod": { - "start": "2018-10-05T09:41:37-07:00", - "end": "2018-10-05T09:56:37-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1", - "resource": { - "resourceType": "Encounter", - "id": "d546dd8a-a78c-4aa8-a99e-74a15f7b88f1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:37e44604-1853-4981-96c4-e191c83cf6d5", - "resource": { - "resourceType": "Observation", - "id": "37e44604-1853-4981-96c4-e191c83cf6d5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 167.1, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:06af490c-2c29-4faf-b826-2167c92aaddf", - "resource": { - "resourceType": "Observation", - "id": "06af490c-2c29-4faf-b826-2167c92aaddf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9a50148c-562b-43e4-8d61-9634b548be88", - "resource": { - "resourceType": "Observation", - "id": "9a50148c-562b-43e4-8d61-9634b548be88", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 85.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b2a5179-75a4-4778-bc44-661f1b62081f", - "resource": { - "resourceType": "Observation", - "id": "3b2a5179-75a4-4778-bc44-661f1b62081f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 30.56, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40589108-8442-4dfe-8bb8-4108695c07c3", - "resource": { - "resourceType": "Observation", - "id": "40589108-8442-4dfe-8bb8-4108695c07c3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 98.13, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e1e077a3-8241-4ad7-a0b4-a4a6ae0c5fe6", - "resource": { - "resourceType": "Observation", - "id": "e1e077a3-8241-4ad7-a0b4-a4a6ae0c5fe6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 124, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:299a9fbc-f0bd-4dfe-a307-d740cfbbbb36", - "resource": { - "resourceType": "Observation", - "id": "299a9fbc-f0bd-4dfe-a307-d740cfbbbb36", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 77, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:faace313-4b23-4b02-ad30-3d319503709c", - "resource": { - "resourceType": "Observation", - "id": "faace313-4b23-4b02-ad30-3d319503709c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:997e98bd-14cc-4593-98b4-8efbe0a1995a", - "resource": { - "resourceType": "Observation", - "id": "997e98bd-14cc-4593-98b4-8efbe0a1995a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "effectiveDateTime": "2019-02-04T08:41:37-08:00", - "issued": "2019-02-04T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8839553d-b467-4604-a905-0a54bff6127c", - "resource": { - "resourceType": "Procedure", - "id": "8839553d-b467-4604-a905-0a54bff6127c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "performedPeriod": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T08:56:37-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c2d91b68-558e-407f-958e-14fdb79c13e9", - "resource": { - "resourceType": "Immunization", - "id": "c2d91b68-558e-407f-958e-14fdb79c13e9", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - }, - "occurrenceDateTime": "2019-02-04T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4d860d59-230f-408f-a8bd-d5603283fd7e", - "resource": { - "resourceType": "Claim", - "id": "4d860d59-230f-408f-a8bd-d5603283fd7e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "created": "2019-02-04T09:11:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c2d91b68-558e-407f-958e-14fdb79c13e9" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:8839553d-b467-4604-a905-0a54bff6127c" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 747.00, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:00dbcc1e-4d4a-4372-b048-1fc72717b155", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "00dbcc1e-4d4a-4372-b048-1fc72717b155", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4d860d59-230f-408f-a8bd-d5603283fd7e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2019-02-04T09:11:37-08:00", - "end": "2020-02-04T09:11:37-08:00" - }, - "created": "2019-02-04T09:11:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4d860d59-230f-408f-a8bd-d5603283fd7e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d546dd8a-a78c-4aa8-a99e-74a15f7b88f1" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2019-02-04T08:41:37-08:00", - "end": "2019-02-04T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 747.00, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 149.4, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 597.6, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 747.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 747.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 710.0160000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48", - "resource": { - "resourceType": "Encounter", - "id": "1187f024-a959-4ab6-87b5-8ffe5af07b48", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "individual": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "display": "Dr. Bertie593 Mante251" - } - } - ], - "period": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d7b4d6c6-b9db-497e-9406-fc249ad26929", - "resource": { - "resourceType": "Observation", - "id": "d7b4d6c6-b9db-497e-9406-fc249ad26929", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 170.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f9400dde-b562-4673-9e42-1d3c3fb7174f", - "resource": { - "resourceType": "Observation", - "id": "f9400dde-b562-4673-9e42-1d3c3fb7174f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:904aac17-b1a0-4e08-8856-cd49813d62a9", - "resource": { - "resourceType": "Observation", - "id": "904aac17-b1a0-4e08-8856-cd49813d62a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 79.4, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ea6e3c7-072c-4c64-af04-a182ab3d4de1", - "resource": { - "resourceType": "Observation", - "id": "9ea6e3c7-072c-4c64-af04-a182ab3d4de1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 27.31, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5b41f610-059c-4cbe-81c6-2d273903a7a7", - "resource": { - "resourceType": "Observation", - "id": "5b41f610-059c-4cbe-81c6-2d273903a7a7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 94.495, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db83d79b-829a-4caf-b4c4-5f75f43e6737", - "resource": { - "resourceType": "Observation", - "id": "db83d79b-829a-4caf-b4c4-5f75f43e6737", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 88, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 102, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:044db950-91d6-410b-9420-7a76fc0be37a", - "resource": { - "resourceType": "Observation", - "id": "044db950-91d6-410b-9420-7a76fc0be37a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 75, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a32d8cb1-80c8-43ba-abfd-c8d0b4d6a915", - "resource": { - "resourceType": "Observation", - "id": "a32d8cb1-80c8-43ba-abfd-c8d0b4d6a915", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", - "resource": { - "resourceType": "Observation", - "id": "3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 8.7281, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28bb65d6-cb96-4d98-b4f2-335ace569991", - "resource": { - "resourceType": "Observation", - "id": "28bb65d6-cb96-4d98-b4f2-335ace569991", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 4.8239, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", - "resource": { - "resourceType": "Observation", - "id": "b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 12.454, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a45d891-738f-4a86-85ba-e97c718ed4b2", - "resource": { - "resourceType": "Observation", - "id": "1a45d891-738f-4a86-85ba-e97c718ed4b2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 36.323, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5cf721e8-9e25-470a-a19a-0389b4b12213", - "resource": { - "resourceType": "Observation", - "id": "5cf721e8-9e25-470a-a19a-0389b4b12213", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 84.472, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:514bd901-b8ac-4871-8b99-d86db24273a9", - "resource": { - "resourceType": "Observation", - "id": "514bd901-b8ac-4871-8b99-d86db24273a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 31.404, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:07f26a58-f3b6-46d1-8d4c-88927bf98ebb", - "resource": { - "resourceType": "Observation", - "id": "07f26a58-f3b6-46d1-8d4c-88927bf98ebb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 34.657, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4174786-f2ce-45cb-bd39-6f70e6d60cb7", - "resource": { - "resourceType": "Observation", - "id": "d4174786-f2ce-45cb-bd39-6f70e6d60cb7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 39.701, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67074975-d413-4fd7-965c-5db34b174ed0", - "resource": { - "resourceType": "Observation", - "id": "67074975-d413-4fd7-965c-5db34b174ed0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 317.45, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71b1e784-b609-4186-9352-32874dbf1fcb", - "resource": { - "resourceType": "Observation", - "id": "71b1e784-b609-4186-9352-32874dbf1fcb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 364.02, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a03734f2-8ff9-4bf3-bb56-e049c409269e", - "resource": { - "resourceType": "Observation", - "id": "a03734f2-8ff9-4bf3-bb56-e049c409269e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueQuantity": { - "value": 11.938, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4bbe85dc-c5f6-4b8b-9f56-ace4e35d7b2a", - "resource": { - "resourceType": "Observation", - "id": "4bbe85dc-c5f6-4b8b-9f56-ace4e35d7b2a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2342c6a-f2b3-49cb-a259-5214d0246872", - "resource": { - "resourceType": "Procedure", - "id": "e2342c6a-f2b3-49cb-a259-5214d0246872", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "performedPeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T08:56:37-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f", - "resource": { - "resourceType": "Immunization", - "id": "0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "occurrenceDateTime": "2020-02-10T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c862abe2-d093-488e-a216-d869790e3d34", - "resource": { - "resourceType": "Immunization", - "id": "c862abe2-d093-488e-a216-d869790e3d34", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "occurrenceDateTime": "2020-02-10T08:41:37-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0404d41c-01f9-4b4e-866d-30d8e1c87768", - "resource": { - "resourceType": "DiagnosticReport", - "id": "0404d41c-01f9-4b4e-866d-30d8e1c87768", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "encounter": { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - }, - "effectiveDateTime": "2020-02-10T08:41:37-08:00", - "issued": "2020-02-10T08:41:37.844-08:00", - "result": [ - { - "reference": "urn:uuid:3efbe489-bdc2-4ea9-b4d1-8d7358968a4d", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:28bb65d6-cb96-4d98-b4f2-335ace569991", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b5cffdcb-43ab-45dc-a72f-9cf7a9d73bad", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:1a45d891-738f-4a86-85ba-e97c718ed4b2", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:5cf721e8-9e25-470a-a19a-0389b4b12213", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:514bd901-b8ac-4871-8b99-d86db24273a9", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:07f26a58-f3b6-46d1-8d4c-88927bf98ebb", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:d4174786-f2ce-45cb-bd39-6f70e6d60cb7", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:67074975-d413-4fd7-965c-5db34b174ed0", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:71b1e784-b609-4186-9352-32874dbf1fcb", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:a03734f2-8ff9-4bf3-bb56-e049c409269e", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:faa1c861-d568-4d37-a9be-e14f02aabbe5", - "resource": { - "resourceType": "Claim", - "id": "faa1c861-d568-4d37-a9be-e14f02aabbe5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4", - "display": "Norberto865 Cole117" - }, - "billablePeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "created": "2020-02-10T09:11:37-08:00", - "provider": { - "reference": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "display": "PCP235727" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:0b3d5ca6-d91d-4185-8bd5-71f05fc9fe2f" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c862abe2-d093-488e-a216-d869790e3d34" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e2342c6a-f2b3-49cb-a259-5214d0246872" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 730.56, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c6949f8d-cb69-4a44-9186-8b336f06dfd4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c6949f8d-cb69-4a44-9186-8b336f06dfd4", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "requester": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "performer": [ - { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "faa1c861-d568-4d37-a9be-e14f02aabbe5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:a5e08ef9-fd71-4273-a53a-d5f5df7926f4" - }, - "billablePeriod": { - "start": "2020-02-10T09:11:37-08:00", - "end": "2021-02-10T09:11:37-08:00" - }, - "created": "2020-02-10T09:11:37-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:faa1c861-d568-4d37-a9be-e14f02aabbe5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - }, - "servicedPeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1187f024-a959-4ab6-87b5-8ffe5af07b48" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "servicedPeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2020-02-10T08:41:37-08:00", - "end": "2020-02-10T09:11:37-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 730.56, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 146.112, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 584.448, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 730.56, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 730.56, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 809.28, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json deleted file mode 100644 index 833a083e3d58..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Robbyn526_DuBuque211_fcf2f472-77ac-47d7-9c9a-a6702bd2bb80.json +++ /dev/null @@ -1,80348 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "resource": { - "resourceType": "Patient", - "id": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -8362707934077662551 Population seed: 1586368870505
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Clelia416 Schmitt836" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", - "valueAddress": { - "city": "Fall River", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 2.806179064509155 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 70.19382093549085 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "SS", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-23-6926" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99931018" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "PPN", - "display": "Passport Number" - } - ], - "text": "Passport Number" - }, - "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", - "value": "X38526821X" - } - ], - "name": [ - { - "use": "official", - "family": "DuBuque211", - "given": [ - "Robbyn526" - ], - "prefix": [ - "Mrs." - ] - }, - { - "use": "maiden", - "family": "Herzog843", - "given": [ - "Robbyn526" - ], - "prefix": [ - "Mrs." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-953-9961", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1946-12-09", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 41.7460901626166 - }, - { - "url": "longitude", - "valueDecimal": -70.7291724846944 - } - ] - } - ], - "line": [ - "149 Conroy Corner Unit 5" - ], - "city": "Wareham", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", - "code": "M", - "display": "M" - } - ], - "text": "M" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "resource": { - "resourceType": "Organization", - "id": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP128586", - "address": [ - { - "line": [ - "2360 CRANBERRY HWY" - ], - "city": "W WAREHAM", - "state": "MA", - "postalCode": "02576-1208", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "resource": { - "resourceType": "Practitioner", - "id": "cf99bf43-51e5-3bab-8623-211cd48f66dc", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999951739" - } - ], - "active": true, - "name": [ - { - "family": "DuBuque211", - "given": [ - "Carlyn477" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Carlyn477.DuBuque211@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "2360 CRANBERRY HWY" - ], - "city": "W WAREHAM", - "state": "MA", - "postalCode": "02576-1208", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3", - "resource": { - "resourceType": "Encounter", - "id": "24d07440-757c-4b8a-9df9-c8bf9e70a6d3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1965-02-01T04:15:51-08:00", - "end": "1965-02-01T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1965-02-01T04:15:51-08:00", - "end": "1965-02-01T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad", - "resource": { - "resourceType": "Condition", - "id": "023eafd5-a008-4f7e-b974-8e2a25e9e3ad", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "59621000", - "display": "Hypertension" - } - ], - "text": "Hypertension" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" - }, - "onsetDateTime": "1965-02-01T04:15:51-08:00", - "recordedDate": "1965-02-01T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:a7e673c4-df3d-4141-8541-9d494b613bb9", - "resource": { - "resourceType": "CareTeam", - "id": "a7e673c4-df3d-4141-8541-9d494b613bb9", - "status": "active", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" - }, - "period": { - "start": "1965-02-01T04:15:51-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "59621000", - "display": "Hypertension" - } - ], - "text": "Hypertension" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:b10b776c-fe84-4b21-9367-8e79d5b2d945", - "resource": { - "resourceType": "CarePlan", - "id": "b10b776c-fe84-4b21-9367-8e79d5b2d945", - "text": { - "status": "generated", - "div": "
    Care Plan for Lifestyle education regarding hypertension.
    Activities:
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension
    • Lifestyle education regarding hypertension

    Care plan is meant to treat Hypertension.
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "443402002", - "display": "Lifestyle education regarding hypertension" - } - ], - "text": "Lifestyle education regarding hypertension" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" - }, - "period": { - "start": "1965-02-01T04:15:51-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:a7e673c4-df3d-4141-8541-9d494b613bb9" - } - ], - "addresses": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "386463000", - "display": "Prescribed activity/exercise education" - } - ], - "text": "Prescribed activity/exercise education" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "413473000", - "display": "Counseling about alcohol consumption" - } - ], - "text": "Counseling about alcohol consumption" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "1151000175103", - "display": "Dietary approaches to stop hypertension diet" - } - ], - "text": "Dietary approaches to stop hypertension diet" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225323000", - "display": "Smoking cessation education" - } - ], - "text": "Smoking cessation education" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:ff4fe9f1-2aef-499b-a72b-809b0d83f747", - "resource": { - "resourceType": "Claim", - "id": "ff4fe9f1-2aef-499b-a72b-809b0d83f747", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1965-02-01T04:15:51-08:00", - "end": "1965-02-01T04:45:51-08:00" - }, - "created": "1965-02-01T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "59621000", - "display": "Hypertension" - } - ], - "text": "Hypertension" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ecebe2d4-1fc3-4819-a5e0-99d6592f74ce", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ecebe2d4-1fc3-4819-a5e0-99d6592f74ce", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Anthem" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Anthem" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ff4fe9f1-2aef-499b-a72b-809b0d83f747" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1965-02-01T04:45:51-08:00", - "end": "1966-02-01T04:45:51-08:00" - }, - "created": "1965-02-01T04:45:51-08:00", - "insurer": { - "display": "Anthem" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ff4fe9f1-2aef-499b-a72b-809b0d83f747" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1965-02-01T04:15:51-08:00", - "end": "1965-02-01T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:24d07440-757c-4b8a-9df9-c8bf9e70a6d3" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "59621000", - "display": "Hypertension" - } - ], - "text": "Hypertension" - }, - "servicedPeriod": { - "start": "1965-02-01T04:15:51-08:00", - "end": "1965-02-01T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "resource": { - "resourceType": "Organization", - "id": "ef6ab57c-ed94-3dbe-9861-812d515918b3", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ef6ab57c-ed94-3dbe-9861-812d515918b3" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAPE COD HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5087711800" - } - ], - "address": [ - { - "line": [ - "88 LEWIS BAY ROAD" - ], - "city": "HYANNIS", - "state": "MA", - "postalCode": "02601", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "resource": { - "resourceType": "Practitioner", - "id": "fb3b949e-e3cb-3069-b280-521ac0512d2e", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999949" - } - ], - "active": true, - "name": [ - { - "family": "Kautzer186", - "given": [ - "Heath320" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Heath320.Kautzer186@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "88 LEWIS BAY ROAD" - ], - "city": "HYANNIS", - "state": "MA", - "postalCode": "02601", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9", - "resource": { - "resourceType": "Encounter", - "id": "4e1a4ce2-7a3c-404f-b744-14309ed21fb9", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1965-03-03T04:15:51-08:00", - "end": "1965-03-03T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1965-03-03T04:15:51-08:00", - "end": "1965-03-03T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0f0e3847-3669-4b37-b635-4ffb0c090bab", - "resource": { - "resourceType": "MedicationRequest", - "id": "0f0e3847-3669-4b37-b635-4ffb0c090bab", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" - }, - "authoredOn": "1965-03-03T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:acf2a21e-53ba-4eea-9976-d6f51576cf10", - "resource": { - "resourceType": "Claim", - "id": "acf2a21e-53ba-4eea-9976-d6f51576cf10", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1965-03-03T04:15:51-08:00", - "end": "1965-03-03T04:30:51-08:00" - }, - "created": "1965-03-03T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0f0e3847-3669-4b37-b635-4ffb0c090bab" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - }, - "encounter": [ - { - "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0a11b721-5c8a-4b1b-a41c-6619a256c242", - "resource": { - "resourceType": "Claim", - "id": "0a11b721-5c8a-4b1b-a41c-6619a256c242", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1965-03-03T04:15:51-08:00", - "end": "1965-03-03T04:30:51-08:00" - }, - "created": "1965-03-03T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - }, - "encounter": [ - { - "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:607389a1-f751-4937-ac5c-bd61e8792def", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "607389a1-f751-4937-ac5c-bd61e8792def", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0a11b721-5c8a-4b1b-a41c-6619a256c242" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1965-03-03T04:30:51-08:00", - "end": "1966-03-03T04:30:51-08:00" - }, - "created": "1965-03-03T04:30:51-08:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0a11b721-5c8a-4b1b-a41c-6619a256c242" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - }, - "servicedPeriod": { - "start": "1965-03-03T04:15:51-08:00", - "end": "1965-03-03T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4e1a4ce2-7a3c-404f-b744-14309ed21fb9" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c", - "resource": { - "resourceType": "Encounter", - "id": "754fecdd-c8c2-4387-9e98-e3680ce6960c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1966-02-07T04:15:51-08:00", - "end": "1966-02-07T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1966-02-07T04:15:51-08:00", - "end": "1966-02-07T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:861d1251-de70-41ec-b9ee-d11ccc175f8f", - "resource": { - "resourceType": "MedicationRequest", - "id": "861d1251-de70-41ec-b9ee-d11ccc175f8f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" - }, - "authoredOn": "1966-02-07T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:7d3303e6-8f8d-4b60-9440-cebd425c778e", - "resource": { - "resourceType": "Claim", - "id": "7d3303e6-8f8d-4b60-9440-cebd425c778e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1966-02-07T04:15:51-08:00", - "end": "1966-02-07T04:45:51-08:00" - }, - "created": "1966-02-07T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:861d1251-de70-41ec-b9ee-d11ccc175f8f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:03abe32d-1671-47a7-ab71-11ca6b50668c", - "resource": { - "resourceType": "Claim", - "id": "03abe32d-1671-47a7-ab71-11ca6b50668c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1966-02-07T04:15:51-08:00", - "end": "1966-02-07T04:45:51-08:00" - }, - "created": "1966-02-07T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ca9a77aa-90f8-4c7c-bbd1-5776630b6f9f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ca9a77aa-90f8-4c7c-bbd1-5776630b6f9f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Anthem" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Anthem" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "03abe32d-1671-47a7-ab71-11ca6b50668c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1966-02-07T04:45:51-08:00", - "end": "1967-02-07T04:45:51-08:00" - }, - "created": "1966-02-07T04:45:51-08:00", - "insurer": { - "display": "Anthem" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:03abe32d-1671-47a7-ab71-11ca6b50668c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1966-02-07T04:15:51-08:00", - "end": "1966-02-07T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:754fecdd-c8c2-4387-9e98-e3680ce6960c" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0", - "resource": { - "resourceType": "Encounter", - "id": "692ff055-48d8-40d0-a4d7-7b3df71a0cb0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1967-02-13T04:15:51-08:00", - "end": "1967-02-13T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1967-02-13T04:15:51-08:00", - "end": "1967-02-13T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7be2a83b-078e-475f-90f0-943b67406ea8", - "resource": { - "resourceType": "MedicationRequest", - "id": "7be2a83b-078e-475f-90f0-943b67406ea8", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" - }, - "authoredOn": "1967-02-13T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:358d194c-0f8f-47a1-925f-960a99387816", - "resource": { - "resourceType": "Claim", - "id": "358d194c-0f8f-47a1-925f-960a99387816", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1967-02-13T04:15:51-08:00", - "end": "1967-02-13T04:30:51-08:00" - }, - "created": "1967-02-13T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:7be2a83b-078e-475f-90f0-943b67406ea8" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2cc2ffea-829c-472f-b75a-e8365aedc49f", - "resource": { - "resourceType": "Claim", - "id": "2cc2ffea-829c-472f-b75a-e8365aedc49f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1967-02-13T04:15:51-08:00", - "end": "1967-02-13T04:30:51-08:00" - }, - "created": "1967-02-13T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e87339f3-9111-4406-b0a8-ed7f05ac82d2", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e87339f3-9111-4406-b0a8-ed7f05ac82d2", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Anthem" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Anthem" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2cc2ffea-829c-472f-b75a-e8365aedc49f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1967-02-13T04:30:51-08:00", - "end": "1968-02-13T04:30:51-08:00" - }, - "created": "1967-02-13T04:30:51-08:00", - "insurer": { - "display": "Anthem" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2cc2ffea-829c-472f-b75a-e8365aedc49f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1967-02-13T04:15:51-08:00", - "end": "1967-02-13T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:692ff055-48d8-40d0-a4d7-7b3df71a0cb0" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4", - "resource": { - "resourceType": "Encounter", - "id": "927ebc18-b85d-4216-8997-85d27747d0f4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1968-02-19T04:15:51-08:00", - "end": "1968-02-19T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1968-02-19T04:15:51-08:00", - "end": "1968-02-19T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0f3ac464-fc33-4fce-bb09-6a8f5b0cc421", - "resource": { - "resourceType": "MedicationRequest", - "id": "0f3ac464-fc33-4fce-bb09-6a8f5b0cc421", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" - }, - "authoredOn": "1968-02-19T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:2ca0b36a-5aa9-4806-9852-457615c05e40", - "resource": { - "resourceType": "Claim", - "id": "2ca0b36a-5aa9-4806-9852-457615c05e40", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1968-02-19T04:15:51-08:00", - "end": "1968-02-19T04:30:51-08:00" - }, - "created": "1968-02-19T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0f3ac464-fc33-4fce-bb09-6a8f5b0cc421" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:992eb68a-753e-42b9-803f-104a349481cf", - "resource": { - "resourceType": "Claim", - "id": "992eb68a-753e-42b9-803f-104a349481cf", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1968-02-19T04:15:51-08:00", - "end": "1968-02-19T04:30:51-08:00" - }, - "created": "1968-02-19T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9708a45f-003e-47db-a707-d9f655b08980", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9708a45f-003e-47db-a707-d9f655b08980", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Anthem" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Anthem" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "992eb68a-753e-42b9-803f-104a349481cf" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1968-02-19T04:30:51-08:00", - "end": "1969-02-19T04:30:51-08:00" - }, - "created": "1968-02-19T04:30:51-08:00", - "insurer": { - "display": "Anthem" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:992eb68a-753e-42b9-803f-104a349481cf" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1968-02-19T04:15:51-08:00", - "end": "1968-02-19T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:927ebc18-b85d-4216-8997-85d27747d0f4" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd", - "resource": { - "resourceType": "Encounter", - "id": "68bc47dc-5742-45b9-be21-55f0219512dd", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1969-02-24T04:15:51-08:00", - "end": "1969-02-24T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1969-02-24T04:15:51-08:00", - "end": "1969-02-24T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8c3c0828-d651-4585-ac06-19de5acf9c03", - "resource": { - "resourceType": "MedicationRequest", - "id": "8c3c0828-d651-4585-ac06-19de5acf9c03", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" - }, - "authoredOn": "1969-02-24T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:cb0963df-89fa-4953-b29c-b858683f523f", - "resource": { - "resourceType": "Claim", - "id": "cb0963df-89fa-4953-b29c-b858683f523f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1969-02-24T04:15:51-08:00", - "end": "1969-02-24T04:45:51-08:00" - }, - "created": "1969-02-24T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:8c3c0828-d651-4585-ac06-19de5acf9c03" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4", - "resource": { - "resourceType": "Claim", - "id": "75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1969-02-24T04:15:51-08:00", - "end": "1969-02-24T04:45:51-08:00" - }, - "created": "1969-02-24T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2f6fff0d-12d7-4a29-baca-f92f9cc4ca21", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2f6fff0d-12d7-4a29-baca-f92f9cc4ca21", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Anthem" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Anthem" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1969-02-24T04:45:51-08:00", - "end": "1970-02-24T04:45:51-08:00" - }, - "created": "1969-02-24T04:45:51-08:00", - "insurer": { - "display": "Anthem" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:75a7ddb9-8f3e-4336-b9cf-1dc81d00d4d4" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Anthem" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1969-02-24T04:15:51-08:00", - "end": "1969-02-24T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:68bc47dc-5742-45b9-be21-55f0219512dd" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c", - "resource": { - "resourceType": "Encounter", - "id": "b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1970-03-02T04:15:51-08:00", - "end": "1970-03-02T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1970-03-02T04:15:51-08:00", - "end": "1970-03-02T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:45640fcb-44b4-4d63-8137-4a3e244083b2", - "resource": { - "resourceType": "MedicationRequest", - "id": "45640fcb-44b4-4d63-8137-4a3e244083b2", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" - }, - "authoredOn": "1970-03-02T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:6b632566-b7a6-4e62-b64e-6a0408b5c9ff", - "resource": { - "resourceType": "Claim", - "id": "6b632566-b7a6-4e62-b64e-6a0408b5c9ff", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1970-03-02T04:15:51-08:00", - "end": "1970-03-02T04:45:51-08:00" - }, - "created": "1970-03-02T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:45640fcb-44b4-4d63-8137-4a3e244083b2" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ffbb426e-ce1d-4bd6-bd42-a4938cb19d04", - "resource": { - "resourceType": "Claim", - "id": "ffbb426e-ce1d-4bd6-bd42-a4938cb19d04", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1970-03-02T04:15:51-08:00", - "end": "1970-03-02T04:45:51-08:00" - }, - "created": "1970-03-02T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:346a9e8a-bafc-4729-a31d-00670c0933ec", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "346a9e8a-bafc-4729-a31d-00670c0933ec", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ffbb426e-ce1d-4bd6-bd42-a4938cb19d04" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1970-03-02T04:45:51-08:00", - "end": "1971-03-02T04:45:51-08:00" - }, - "created": "1970-03-02T04:45:51-08:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ffbb426e-ce1d-4bd6-bd42-a4938cb19d04" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1970-03-02T04:15:51-08:00", - "end": "1970-03-02T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b9bb4755-e2ae-4c5f-9db3-a0d4f08ab35c" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163", - "resource": { - "resourceType": "Encounter", - "id": "0d03108a-c9c2-4134-99b1-6a58b8d61163", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1971-03-08T04:15:51-08:00", - "end": "1971-03-08T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1971-03-08T04:15:51-08:00", - "end": "1971-03-08T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f68524e7-b47e-4495-95cd-c18426e25643", - "resource": { - "resourceType": "MedicationRequest", - "id": "f68524e7-b47e-4495-95cd-c18426e25643", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" - }, - "authoredOn": "1971-03-08T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:bfb8ea92-2de4-4f19-9bcb-0b6e1080035b", - "resource": { - "resourceType": "Claim", - "id": "bfb8ea92-2de4-4f19-9bcb-0b6e1080035b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1971-03-08T04:15:51-08:00", - "end": "1971-03-08T04:30:51-08:00" - }, - "created": "1971-03-08T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:f68524e7-b47e-4495-95cd-c18426e25643" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:99650392-90a6-4f14-aa47-34fdfe24f274", - "resource": { - "resourceType": "Claim", - "id": "99650392-90a6-4f14-aa47-34fdfe24f274", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1971-03-08T04:15:51-08:00", - "end": "1971-03-08T04:30:51-08:00" - }, - "created": "1971-03-08T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e286c550-3649-46dd-beef-10bf70a33ee3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e286c550-3649-46dd-beef-10bf70a33ee3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "99650392-90a6-4f14-aa47-34fdfe24f274" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1971-03-08T04:30:51-08:00", - "end": "1972-03-08T04:30:51-08:00" - }, - "created": "1971-03-08T04:30:51-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:99650392-90a6-4f14-aa47-34fdfe24f274" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1971-03-08T04:15:51-08:00", - "end": "1971-03-08T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0d03108a-c9c2-4134-99b1-6a58b8d61163" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc", - "resource": { - "resourceType": "Encounter", - "id": "ef56877c-c52a-4644-9a93-a2ad9a9f08dc", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1972-03-13T04:15:51-08:00", - "end": "1972-03-13T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1972-03-13T04:15:51-08:00", - "end": "1972-03-13T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bbd477f9-bb7f-4571-af62-f3f133e291e1", - "resource": { - "resourceType": "MedicationRequest", - "id": "bbd477f9-bb7f-4571-af62-f3f133e291e1", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" - }, - "authoredOn": "1972-03-13T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8d68d79a-8a7c-4d8e-81ef-8c118a99ca71", - "resource": { - "resourceType": "Claim", - "id": "8d68d79a-8a7c-4d8e-81ef-8c118a99ca71", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-03-13T04:15:51-08:00", - "end": "1972-03-13T04:45:51-08:00" - }, - "created": "1972-03-13T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:bbd477f9-bb7f-4571-af62-f3f133e291e1" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:73a52d4c-ebd0-4887-8677-e33e3a98d116", - "resource": { - "resourceType": "Claim", - "id": "73a52d4c-ebd0-4887-8677-e33e3a98d116", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1972-03-13T04:15:51-08:00", - "end": "1972-03-13T04:45:51-08:00" - }, - "created": "1972-03-13T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:76a827e3-25b7-4ae4-b249-58b8fb79fb33", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "76a827e3-25b7-4ae4-b249-58b8fb79fb33", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "73a52d4c-ebd0-4887-8677-e33e3a98d116" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-03-13T04:45:51-08:00", - "end": "1973-03-13T04:45:51-08:00" - }, - "created": "1972-03-13T04:45:51-08:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:73a52d4c-ebd0-4887-8677-e33e3a98d116" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1972-03-13T04:15:51-08:00", - "end": "1972-03-13T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ef56877c-c52a-4644-9a93-a2ad9a9f08dc" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7", - "resource": { - "resourceType": "Encounter", - "id": "c47302e4-bb57-4490-a42e-e01c90ccf8f7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1972-09-18T05:15:51-07:00", - "end": "1972-09-18T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1972-09-18T05:15:51-07:00", - "end": "1972-09-18T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a7cfc82f-273e-45cb-a6ea-0ed1320fc709", - "resource": { - "resourceType": "MedicationRequest", - "id": "a7cfc82f-273e-45cb-a6ea-0ed1320fc709", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" - }, - "authoredOn": "1972-09-18T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:af048b54-7375-4aa0-8d13-06670e09747f", - "resource": { - "resourceType": "Claim", - "id": "af048b54-7375-4aa0-8d13-06670e09747f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-09-18T05:15:51-07:00", - "end": "1972-09-18T05:30:51-07:00" - }, - "created": "1972-09-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:a7cfc82f-273e-45cb-a6ea-0ed1320fc709" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c49ee98c-d8b1-4599-8020-d2aaeed2999c", - "resource": { - "resourceType": "Claim", - "id": "c49ee98c-d8b1-4599-8020-d2aaeed2999c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1972-09-18T05:15:51-07:00", - "end": "1972-09-18T05:30:51-07:00" - }, - "created": "1972-09-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5b904601-474f-4285-8282-b0b9e147f594", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5b904601-474f-4285-8282-b0b9e147f594", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c49ee98c-d8b1-4599-8020-d2aaeed2999c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-09-18T05:30:51-07:00", - "end": "1973-09-18T05:30:51-07:00" - }, - "created": "1972-09-18T05:30:51-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:c49ee98c-d8b1-4599-8020-d2aaeed2999c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "1972-09-18T05:15:51-07:00", - "end": "1972-09-18T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c47302e4-bb57-4490-a42e-e01c90ccf8f7" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8", - "resource": { - "resourceType": "Encounter", - "id": "220a9e60-1006-4228-b2ae-42375684add8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1972-09-13T05:15:51-07:00", - "end": "1972-09-13T05:40:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1972-09-13T05:15:51-07:00", - "end": "1972-09-13T05:40:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a25fd790-0bec-4227-8b14-c0cd89a034f0", - "resource": { - "resourceType": "MedicationRequest", - "id": "a25fd790-0bec-4227-8b14-c0cd89a034f0", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" - }, - "authoredOn": "1972-09-18T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:69eb6011-54c3-49db-b92c-3a2cf10a3273", - "resource": { - "resourceType": "Claim", - "id": "69eb6011-54c3-49db-b92c-3a2cf10a3273", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-09-13T05:15:51-07:00", - "end": "1972-09-13T05:40:51-07:00" - }, - "created": "1972-09-13T05:40:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:a25fd790-0bec-4227-8b14-c0cd89a034f0" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0fac4e42-6633-4d77-9c45-9c83356280d9", - "resource": { - "resourceType": "Claim", - "id": "0fac4e42-6633-4d77-9c45-9c83356280d9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1972-09-13T05:15:51-07:00", - "end": "1972-09-13T05:40:51-07:00" - }, - "created": "1972-09-13T05:40:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:29138263-4c9d-4002-9ee3-b532f5c0d244", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "29138263-4c9d-4002-9ee3-b532f5c0d244", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Cigna Health" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Cigna Health" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0fac4e42-6633-4d77-9c45-9c83356280d9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1972-09-13T05:40:51-07:00", - "end": "1973-09-13T05:40:51-07:00" - }, - "created": "1972-09-13T05:40:51-07:00", - "insurer": { - "display": "Cigna Health" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0fac4e42-6633-4d77-9c45-9c83356280d9" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Cigna Health" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "1972-09-13T05:15:51-07:00", - "end": "1972-09-13T05:40:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:220a9e60-1006-4228-b2ae-42375684add8" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379", - "resource": { - "resourceType": "Encounter", - "id": "305d11c5-6d30-49e7-87cf-35dbd794c379", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1973-03-19T04:15:51-08:00", - "end": "1973-03-19T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1973-03-19T04:15:51-08:00", - "end": "1973-03-19T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9", - "resource": { - "resourceType": "MedicationRequest", - "id": "d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" - }, - "authoredOn": "1973-03-19T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:abda0fee-3f27-4461-b3f5-5cd1bdb20605", - "resource": { - "resourceType": "Claim", - "id": "abda0fee-3f27-4461-b3f5-5cd1bdb20605", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1973-03-19T04:15:51-08:00", - "end": "1973-03-19T04:30:51-08:00" - }, - "created": "1973-03-19T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:d5bb1aba-1b9c-4f83-87fc-ff7bffe0daa9" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ac30b172-6737-4dbf-8abf-28bad6eb19d3", - "resource": { - "resourceType": "Claim", - "id": "ac30b172-6737-4dbf-8abf-28bad6eb19d3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1973-03-19T04:15:51-08:00", - "end": "1973-03-19T04:30:51-08:00" - }, - "created": "1973-03-19T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e41d7307-5b9b-40ef-867c-a70e20259fab", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e41d7307-5b9b-40ef-867c-a70e20259fab", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ac30b172-6737-4dbf-8abf-28bad6eb19d3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1973-03-19T04:30:51-08:00", - "end": "1974-03-19T04:30:51-07:00" - }, - "created": "1973-03-19T04:30:51-08:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ac30b172-6737-4dbf-8abf-28bad6eb19d3" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1973-03-19T04:15:51-08:00", - "end": "1973-03-19T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:305d11c5-6d30-49e7-87cf-35dbd794c379" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed", - "resource": { - "resourceType": "Encounter", - "id": "8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1974-03-25T05:15:51-07:00", - "end": "1974-03-25T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1974-03-25T05:15:51-07:00", - "end": "1974-03-25T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d5557008-cbad-4d0b-8be0-f70608079eac", - "resource": { - "resourceType": "MedicationRequest", - "id": "d5557008-cbad-4d0b-8be0-f70608079eac", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" - }, - "authoredOn": "1974-03-25T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0e214bf1-1d2f-4b69-b2ff-8c9693a41fdb", - "resource": { - "resourceType": "Claim", - "id": "0e214bf1-1d2f-4b69-b2ff-8c9693a41fdb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1974-03-25T05:15:51-07:00", - "end": "1974-03-25T05:45:51-07:00" - }, - "created": "1974-03-25T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:d5557008-cbad-4d0b-8be0-f70608079eac" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:466edc12-b2eb-433c-b1c7-deb51aa2d319", - "resource": { - "resourceType": "Claim", - "id": "466edc12-b2eb-433c-b1c7-deb51aa2d319", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1974-03-25T05:15:51-07:00", - "end": "1974-03-25T05:45:51-07:00" - }, - "created": "1974-03-25T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1ddd9d3a-2a99-4c89-9b95-24081772e1ca", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1ddd9d3a-2a99-4c89-9b95-24081772e1ca", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "466edc12-b2eb-433c-b1c7-deb51aa2d319" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1974-03-25T05:45:51-07:00", - "end": "1975-03-25T05:45:51-07:00" - }, - "created": "1974-03-25T05:45:51-07:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:466edc12-b2eb-433c-b1c7-deb51aa2d319" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1974-03-25T05:15:51-07:00", - "end": "1974-03-25T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:8d4b1dc2-e4ff-43d2-8d26-f751d289e2ed" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66", - "resource": { - "resourceType": "Encounter", - "id": "57fee96f-a86d-4544-afa1-f6f6c8e37d66", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1975-03-31T05:15:51-07:00", - "end": "1975-03-31T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1975-03-31T05:15:51-07:00", - "end": "1975-03-31T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:82eaade0-3a05-4290-9324-9c4c2b0473bf", - "resource": { - "resourceType": "MedicationRequest", - "id": "82eaade0-3a05-4290-9324-9c4c2b0473bf", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" - }, - "authoredOn": "1975-03-31T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:c687d65a-2f8a-46e1-b8b5-ed89d1852087", - "resource": { - "resourceType": "Claim", - "id": "c687d65a-2f8a-46e1-b8b5-ed89d1852087", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1975-03-31T05:15:51-07:00", - "end": "1975-03-31T05:30:51-07:00" - }, - "created": "1975-03-31T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:82eaade0-3a05-4290-9324-9c4c2b0473bf" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e0bb258a-6080-4f6a-b2d3-7fbe8928eace", - "resource": { - "resourceType": "Claim", - "id": "e0bb258a-6080-4f6a-b2d3-7fbe8928eace", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1975-03-31T05:15:51-07:00", - "end": "1975-03-31T05:30:51-07:00" - }, - "created": "1975-03-31T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b152873b-f16d-40a8-92db-f2a3f42607fa", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b152873b-f16d-40a8-92db-f2a3f42607fa", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e0bb258a-6080-4f6a-b2d3-7fbe8928eace" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1975-03-31T05:30:51-07:00", - "end": "1976-03-31T05:30:51-08:00" - }, - "created": "1975-03-31T05:30:51-07:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e0bb258a-6080-4f6a-b2d3-7fbe8928eace" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1975-03-31T05:15:51-07:00", - "end": "1975-03-31T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:57fee96f-a86d-4544-afa1-f6f6c8e37d66" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e", - "resource": { - "resourceType": "Encounter", - "id": "35707937-d9c0-4f0b-824f-8f40abfccf9e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1976-04-05T04:15:51-08:00", - "end": "1976-04-05T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1976-04-05T04:15:51-08:00", - "end": "1976-04-05T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:33efd06d-3665-4930-82d7-5ecd7fc95b20", - "resource": { - "resourceType": "MedicationRequest", - "id": "33efd06d-3665-4930-82d7-5ecd7fc95b20", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" - }, - "authoredOn": "1976-04-05T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5f18df60-3875-4f96-895c-3c7f1e4589b1", - "resource": { - "resourceType": "Claim", - "id": "5f18df60-3875-4f96-895c-3c7f1e4589b1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1976-04-05T04:15:51-08:00", - "end": "1976-04-05T04:45:51-08:00" - }, - "created": "1976-04-05T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:33efd06d-3665-4930-82d7-5ecd7fc95b20" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Blue Cross Blue Shield" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:59c698a6-66c5-4212-a73f-6db2c77a999a", - "resource": { - "resourceType": "Claim", - "id": "59c698a6-66c5-4212-a73f-6db2c77a999a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1976-04-05T04:15:51-08:00", - "end": "1976-04-05T04:45:51-08:00" - }, - "created": "1976-04-05T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Blue Cross Blue Shield" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2651c4a2-d9e0-4aa1-8f0e-d763b1295ea4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2651c4a2-d9e0-4aa1-8f0e-d763b1295ea4", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Blue Cross Blue Shield" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Blue Cross Blue Shield" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "59c698a6-66c5-4212-a73f-6db2c77a999a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1976-04-05T04:45:51-08:00", - "end": "1977-04-05T04:45:51-08:00" - }, - "created": "1976-04-05T04:45:51-08:00", - "insurer": { - "display": "Blue Cross Blue Shield" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:59c698a6-66c5-4212-a73f-6db2c77a999a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Blue Cross Blue Shield" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1976-04-05T04:15:51-08:00", - "end": "1976-04-05T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:35707937-d9c0-4f0b-824f-8f40abfccf9e" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555", - "resource": { - "resourceType": "Encounter", - "id": "d2d91306-a6ad-442e-a83f-6d97844de555", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1977-04-11T04:15:51-08:00", - "end": "1977-04-11T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1977-04-11T04:15:51-08:00", - "end": "1977-04-11T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:75768d8c-5e38-4a02-9e32-b578a957abe1", - "resource": { - "resourceType": "MedicationRequest", - "id": "75768d8c-5e38-4a02-9e32-b578a957abe1", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" - }, - "authoredOn": "1977-04-11T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8b7edca7-5f49-4fbb-bbaa-76958659f7be", - "resource": { - "resourceType": "Claim", - "id": "8b7edca7-5f49-4fbb-bbaa-76958659f7be", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1977-04-11T04:15:51-08:00", - "end": "1977-04-11T04:30:51-08:00" - }, - "created": "1977-04-11T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:75768d8c-5e38-4a02-9e32-b578a957abe1" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:636ac4e4-059b-4dc9-a38c-66f24f46a47c", - "resource": { - "resourceType": "Claim", - "id": "636ac4e4-059b-4dc9-a38c-66f24f46a47c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1977-04-11T04:15:51-08:00", - "end": "1977-04-11T04:30:51-08:00" - }, - "created": "1977-04-11T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fda88127-d697-4d34-9f77-b5adf3381a17", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "fda88127-d697-4d34-9f77-b5adf3381a17", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "636ac4e4-059b-4dc9-a38c-66f24f46a47c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1977-04-11T04:30:51-08:00", - "end": "1978-04-11T04:30:51-08:00" - }, - "created": "1977-04-11T04:30:51-08:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:636ac4e4-059b-4dc9-a38c-66f24f46a47c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1977-04-11T04:15:51-08:00", - "end": "1977-04-11T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d2d91306-a6ad-442e-a83f-6d97844de555" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217", - "resource": { - "resourceType": "Encounter", - "id": "1514ab4e-4d9f-4518-8364-e8af6bcc8217", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1978-04-17T04:15:51-08:00", - "end": "1978-04-17T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1978-04-17T04:15:51-08:00", - "end": "1978-04-17T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:db6b3a0f-d791-48cf-8a81-5601a71b2043", - "resource": { - "resourceType": "MedicationRequest", - "id": "db6b3a0f-d791-48cf-8a81-5601a71b2043", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" - }, - "authoredOn": "1978-04-17T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8fa05e2c-fddd-46ff-9357-1e5c02ddb766", - "resource": { - "resourceType": "Claim", - "id": "8fa05e2c-fddd-46ff-9357-1e5c02ddb766", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1978-04-17T04:15:51-08:00", - "end": "1978-04-17T04:45:51-08:00" - }, - "created": "1978-04-17T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:db6b3a0f-d791-48cf-8a81-5601a71b2043" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:36e36508-cae4-461c-bbd0-6bbb7a0a4dea", - "resource": { - "resourceType": "Claim", - "id": "36e36508-cae4-461c-bbd0-6bbb7a0a4dea", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1978-04-17T04:15:51-08:00", - "end": "1978-04-17T04:45:51-08:00" - }, - "created": "1978-04-17T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bbef9ec2-58fb-4ccd-ad38-fb140ebfc3bc", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "bbef9ec2-58fb-4ccd-ad38-fb140ebfc3bc", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "36e36508-cae4-461c-bbd0-6bbb7a0a4dea" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1978-04-17T04:45:51-08:00", - "end": "1979-04-17T04:45:51-08:00" - }, - "created": "1978-04-17T04:45:51-08:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:36e36508-cae4-461c-bbd0-6bbb7a0a4dea" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1978-04-17T04:15:51-08:00", - "end": "1978-04-17T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1514ab4e-4d9f-4518-8364-e8af6bcc8217" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188", - "resource": { - "resourceType": "Encounter", - "id": "4f930943-2e18-4fc3-9573-cd2d637e0188", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a", - "resource": { - "resourceType": "Condition", - "id": "7e29d4a9-b5b9-4965-bfa6-02b3ad26418a", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - }, - "onsetDateTime": "1979-04-23T04:15:51-08:00", - "recordedDate": "1979-04-23T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ed07769e-89e0-40df-86e9-48dcbbd2e97d", - "resource": { - "resourceType": "MedicationRequest", - "id": "ed07769e-89e0-40df-86e9-48dcbbd2e97d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - }, - "authoredOn": "1979-04-23T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5c8c256c-9428-49c9-bef9-a78f6a64f2ba", - "resource": { - "resourceType": "Claim", - "id": "5c8c256c-9428-49c9-bef9-a78f6a64f2ba", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "created": "1979-04-23T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ed07769e-89e0-40df-86e9-48dcbbd2e97d" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9", - "resource": { - "resourceType": "CareTeam", - "id": "ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9", - "status": "active", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - }, - "period": { - "start": "1979-04-23T04:15:51-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:26c6277e-cc3a-47bd-8838-fba69ceb3a82", - "resource": { - "resourceType": "CarePlan", - "id": "26c6277e-cc3a-47bd-8838-fba69ceb3a82", - "text": { - "status": "generated", - "div": "
    Care Plan for Diabetes self management plan.
    Activities:
    • Diabetes self management plan
    • Diabetes self management plan

    Care plan is meant to treat Prediabetes.
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698360004", - "display": "Diabetes self management plan" - } - ], - "text": "Diabetes self management plan" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - }, - "period": { - "start": "1979-04-23T04:15:51-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:ce93ebf6-9ad2-409c-81ef-a32ec8bb4ac9" - } - ], - "addresses": [ - { - "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "160670007", - "display": "Diabetic diet" - } - ], - "text": "Diabetic diet" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229065009", - "display": "Exercise therapy" - } - ], - "text": "Exercise therapy" - }, - "status": "in-progress", - "location": { - "display": "PCP128586" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:c3eea979-eb01-406b-952f-76483bb31ca0", - "resource": { - "resourceType": "Claim", - "id": "c3eea979-eb01-406b-952f-76483bb31ca0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "created": "1979-04-23T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:32fd4fad-501a-4730-96a1-371ede601291", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "32fd4fad-501a-4730-96a1-371ede601291", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c3eea979-eb01-406b-952f-76483bb31ca0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1979-04-23T04:30:51-08:00", - "end": "1980-04-23T04:30:51-08:00" - }, - "created": "1979-04-23T04:30:51-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:c3eea979-eb01-406b-952f-76483bb31ca0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7e29d4a9-b5b9-4965-bfa6-02b3ad26418a" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4f930943-2e18-4fc3-9573-cd2d637e0188" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "15777000", - "display": "Prediabetes" - } - ], - "text": "Prediabetes" - }, - "servicedPeriod": { - "start": "1979-04-23T04:15:51-08:00", - "end": "1979-04-23T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9", - "resource": { - "resourceType": "Encounter", - "id": "d96d0c1a-6842-44fe-8e28-20d437a374c9", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1980-04-28T05:15:51-07:00", - "end": "1980-04-28T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1980-04-28T05:15:51-07:00", - "end": "1980-04-28T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0c479a35-4f68-4b13-9193-e3ae5c98d28a", - "resource": { - "resourceType": "MedicationRequest", - "id": "0c479a35-4f68-4b13-9193-e3ae5c98d28a", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" - }, - "authoredOn": "1980-04-28T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:dfce59ce-4bd0-480e-aed4-fa8b355b4fb8", - "resource": { - "resourceType": "Claim", - "id": "dfce59ce-4bd0-480e-aed4-fa8b355b4fb8", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1980-04-28T05:15:51-07:00", - "end": "1980-04-28T05:30:51-07:00" - }, - "created": "1980-04-28T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0c479a35-4f68-4b13-9193-e3ae5c98d28a" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:62a209ad-10fe-412c-ba4d-b605321c027f", - "resource": { - "resourceType": "Claim", - "id": "62a209ad-10fe-412c-ba4d-b605321c027f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1980-04-28T05:15:51-07:00", - "end": "1980-04-28T05:30:51-07:00" - }, - "created": "1980-04-28T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3c55935b-b90c-4ca4-a9a2-442cf767597b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3c55935b-b90c-4ca4-a9a2-442cf767597b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "62a209ad-10fe-412c-ba4d-b605321c027f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1980-04-28T05:30:51-07:00", - "end": "1981-04-28T05:30:51-07:00" - }, - "created": "1980-04-28T05:30:51-07:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:62a209ad-10fe-412c-ba4d-b605321c027f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1980-04-28T05:15:51-07:00", - "end": "1980-04-28T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d96d0c1a-6842-44fe-8e28-20d437a374c9" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39", - "resource": { - "resourceType": "Encounter", - "id": "d172e169-67dd-4311-ac98-c79c0b5a2b39", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1981-05-04T05:15:51-07:00", - "end": "1981-05-04T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1981-05-04T05:15:51-07:00", - "end": "1981-05-04T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:57131a9d-b05a-4844-9981-69d85ae54117", - "resource": { - "resourceType": "MedicationRequest", - "id": "57131a9d-b05a-4844-9981-69d85ae54117", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" - }, - "authoredOn": "1981-05-04T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:af120bbe-ebf6-45f4-ae26-4fd080a91c2c", - "resource": { - "resourceType": "Claim", - "id": "af120bbe-ebf6-45f4-ae26-4fd080a91c2c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1981-05-04T05:15:51-07:00", - "end": "1981-05-04T05:30:51-07:00" - }, - "created": "1981-05-04T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:57131a9d-b05a-4844-9981-69d85ae54117" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:907e442a-2460-465c-9ceb-f4050ac15d9a", - "resource": { - "resourceType": "Claim", - "id": "907e442a-2460-465c-9ceb-f4050ac15d9a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1981-05-04T05:15:51-07:00", - "end": "1981-05-04T05:30:51-07:00" - }, - "created": "1981-05-04T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a62decf1-8628-49df-976e-547142fcc11b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a62decf1-8628-49df-976e-547142fcc11b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicaid" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicaid" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "907e442a-2460-465c-9ceb-f4050ac15d9a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1981-05-04T05:30:51-07:00", - "end": "1982-05-04T05:30:51-07:00" - }, - "created": "1981-05-04T05:30:51-07:00", - "insurer": { - "display": "Medicaid" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:907e442a-2460-465c-9ceb-f4050ac15d9a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicaid" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1981-05-04T05:15:51-07:00", - "end": "1981-05-04T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d172e169-67dd-4311-ac98-c79c0b5a2b39" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321", - "resource": { - "resourceType": "Encounter", - "id": "efd7d1b4-a401-4239-8984-002f8ab6c321", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1982-05-10T05:15:51-07:00", - "end": "1982-05-10T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1982-05-10T05:15:51-07:00", - "end": "1982-05-10T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:06f498c9-eb07-499e-b73e-cd95843f9d51", - "resource": { - "resourceType": "MedicationRequest", - "id": "06f498c9-eb07-499e-b73e-cd95843f9d51", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" - }, - "authoredOn": "1982-05-10T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0aa4a8d2-c79e-4a8f-af81-3af95daf630b", - "resource": { - "resourceType": "Claim", - "id": "0aa4a8d2-c79e-4a8f-af81-3af95daf630b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1982-05-10T05:15:51-07:00", - "end": "1982-05-10T05:30:51-07:00" - }, - "created": "1982-05-10T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:06f498c9-eb07-499e-b73e-cd95843f9d51" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5", - "resource": { - "resourceType": "Claim", - "id": "6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1982-05-10T05:15:51-07:00", - "end": "1982-05-10T05:30:51-07:00" - }, - "created": "1982-05-10T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:97e0e4b6-140f-41d6-86c2-16eec511e6fb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "97e0e4b6-140f-41d6-86c2-16eec511e6fb", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1982-05-10T05:30:51-07:00", - "end": "1983-05-10T05:30:51-07:00" - }, - "created": "1982-05-10T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:6bbb83c9-cb98-42b3-a6c0-ffd23bd9bdc5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1982-05-10T05:15:51-07:00", - "end": "1982-05-10T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:efd7d1b4-a401-4239-8984-002f8ab6c321" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0", - "resource": { - "resourceType": "Encounter", - "id": "083f8d20-2266-40bb-8dad-315bea4de8a0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1983-05-16T05:15:51-07:00", - "end": "1983-05-16T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1983-05-16T05:15:51-07:00", - "end": "1983-05-16T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1f013ce3-f405-4b5f-8572-c6953663f659", - "resource": { - "resourceType": "MedicationRequest", - "id": "1f013ce3-f405-4b5f-8572-c6953663f659", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" - }, - "authoredOn": "1983-05-16T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:3f5e693c-e540-4411-968c-747723cfbe03", - "resource": { - "resourceType": "Claim", - "id": "3f5e693c-e540-4411-968c-747723cfbe03", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1983-05-16T05:15:51-07:00", - "end": "1983-05-16T05:45:51-07:00" - }, - "created": "1983-05-16T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:1f013ce3-f405-4b5f-8572-c6953663f659" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1224c44e-8f7a-4ea6-846c-9adad0fb89c2", - "resource": { - "resourceType": "Claim", - "id": "1224c44e-8f7a-4ea6-846c-9adad0fb89c2", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1983-05-16T05:15:51-07:00", - "end": "1983-05-16T05:45:51-07:00" - }, - "created": "1983-05-16T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:310488a9-7ee4-47dc-9fda-c675b9c47e9c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "310488a9-7ee4-47dc-9fda-c675b9c47e9c", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1224c44e-8f7a-4ea6-846c-9adad0fb89c2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1983-05-16T05:45:51-07:00", - "end": "1984-05-16T05:45:51-07:00" - }, - "created": "1983-05-16T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:1224c44e-8f7a-4ea6-846c-9adad0fb89c2" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1983-05-16T05:15:51-07:00", - "end": "1983-05-16T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:083f8d20-2266-40bb-8dad-315bea4de8a0" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e", - "resource": { - "resourceType": "Encounter", - "id": "f2af01b9-93e5-4aeb-b710-86875c752c9e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1984-05-21T05:15:51-07:00", - "end": "1984-05-21T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1984-05-21T05:15:51-07:00", - "end": "1984-05-21T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0ed9f205-be7a-44cb-b103-7ca50e493292", - "resource": { - "resourceType": "MedicationRequest", - "id": "0ed9f205-be7a-44cb-b103-7ca50e493292", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" - }, - "authoredOn": "1984-05-21T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:3e5c63f7-3be6-453d-b5ef-b7ae74c275c0", - "resource": { - "resourceType": "Claim", - "id": "3e5c63f7-3be6-453d-b5ef-b7ae74c275c0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1984-05-21T05:15:51-07:00", - "end": "1984-05-21T05:45:51-07:00" - }, - "created": "1984-05-21T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0ed9f205-be7a-44cb-b103-7ca50e493292" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c", - "resource": { - "resourceType": "Claim", - "id": "0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1984-05-21T05:15:51-07:00", - "end": "1984-05-21T05:45:51-07:00" - }, - "created": "1984-05-21T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d650b7de-fa34-4e57-aded-1bba643c9e04", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d650b7de-fa34-4e57-aded-1bba643c9e04", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1984-05-21T05:45:51-07:00", - "end": "1985-05-21T05:45:51-07:00" - }, - "created": "1984-05-21T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0dd62618-9caa-4b3b-a65d-44a8ed5dfb9c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1984-05-21T05:15:51-07:00", - "end": "1984-05-21T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f2af01b9-93e5-4aeb-b710-86875c752c9e" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e", - "resource": { - "resourceType": "Encounter", - "id": "6be1a524-887d-4221-8114-df00cdd6e68e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1985-05-27T05:15:51-07:00", - "end": "1985-05-27T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1985-05-27T05:15:51-07:00", - "end": "1985-05-27T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1a1dba2d-3cf7-4391-b54d-c05eff05910b", - "resource": { - "resourceType": "MedicationRequest", - "id": "1a1dba2d-3cf7-4391-b54d-c05eff05910b", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" - }, - "authoredOn": "1985-05-27T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:9232907d-e3f1-452c-ac47-c2f93ac25580", - "resource": { - "resourceType": "Claim", - "id": "9232907d-e3f1-452c-ac47-c2f93ac25580", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1985-05-27T05:15:51-07:00", - "end": "1985-05-27T05:30:51-07:00" - }, - "created": "1985-05-27T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:1a1dba2d-3cf7-4391-b54d-c05eff05910b" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4013d214-b869-445e-9f37-81e756896650", - "resource": { - "resourceType": "Claim", - "id": "4013d214-b869-445e-9f37-81e756896650", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1985-05-27T05:15:51-07:00", - "end": "1985-05-27T05:30:51-07:00" - }, - "created": "1985-05-27T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:605c40b9-5636-4f69-9c46-637cbdf1ca14", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "605c40b9-5636-4f69-9c46-637cbdf1ca14", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4013d214-b869-445e-9f37-81e756896650" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1985-05-27T05:30:51-07:00", - "end": "1986-05-27T05:30:51-07:00" - }, - "created": "1985-05-27T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4013d214-b869-445e-9f37-81e756896650" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1985-05-27T05:15:51-07:00", - "end": "1985-05-27T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:6be1a524-887d-4221-8114-df00cdd6e68e" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf", - "resource": { - "resourceType": "Encounter", - "id": "18acea1a-e5a0-4167-9e6b-b957ceae92cf", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1986-06-02T05:15:51-07:00", - "end": "1986-06-02T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1986-06-02T05:15:51-07:00", - "end": "1986-06-02T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:42fbc707-db93-4efb-bfb9-4d97192d898f", - "resource": { - "resourceType": "MedicationRequest", - "id": "42fbc707-db93-4efb-bfb9-4d97192d898f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" - }, - "authoredOn": "1986-06-02T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d67ee42b-c853-44bb-a039-ead835f1b47b", - "resource": { - "resourceType": "Claim", - "id": "d67ee42b-c853-44bb-a039-ead835f1b47b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1986-06-02T05:15:51-07:00", - "end": "1986-06-02T05:45:51-07:00" - }, - "created": "1986-06-02T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:42fbc707-db93-4efb-bfb9-4d97192d898f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8343277b-1210-4e65-9b64-a5a56a413363", - "resource": { - "resourceType": "Claim", - "id": "8343277b-1210-4e65-9b64-a5a56a413363", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1986-06-02T05:15:51-07:00", - "end": "1986-06-02T05:45:51-07:00" - }, - "created": "1986-06-02T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3ea0421f-f3a3-4293-b82a-254247dc8079", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3ea0421f-f3a3-4293-b82a-254247dc8079", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8343277b-1210-4e65-9b64-a5a56a413363" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1986-06-02T05:45:51-07:00", - "end": "1987-06-02T05:45:51-07:00" - }, - "created": "1986-06-02T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8343277b-1210-4e65-9b64-a5a56a413363" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1986-06-02T05:15:51-07:00", - "end": "1986-06-02T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:18acea1a-e5a0-4167-9e6b-b957ceae92cf" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909", - "resource": { - "resourceType": "Encounter", - "id": "f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1987-06-08T05:15:51-07:00", - "end": "1987-06-08T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1987-06-08T05:15:51-07:00", - "end": "1987-06-08T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:dc0523b1-d74a-4472-9efb-3dfa96584c29", - "resource": { - "resourceType": "MedicationRequest", - "id": "dc0523b1-d74a-4472-9efb-3dfa96584c29", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" - }, - "authoredOn": "1987-06-08T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a0a88256-2b5d-41a5-b14d-a1f55dd3a22b", - "resource": { - "resourceType": "Claim", - "id": "a0a88256-2b5d-41a5-b14d-a1f55dd3a22b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-06-08T05:15:51-07:00", - "end": "1987-06-08T05:45:51-07:00" - }, - "created": "1987-06-08T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:dc0523b1-d74a-4472-9efb-3dfa96584c29" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:99f0bc94-4c7d-43b2-aa7d-a10111086f4c", - "resource": { - "resourceType": "Claim", - "id": "99f0bc94-4c7d-43b2-aa7d-a10111086f4c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1987-06-08T05:15:51-07:00", - "end": "1987-06-08T05:45:51-07:00" - }, - "created": "1987-06-08T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6e2c600e-ebf0-46a1-ae92-732bb31be0af", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6e2c600e-ebf0-46a1-ae92-732bb31be0af", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "99f0bc94-4c7d-43b2-aa7d-a10111086f4c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-06-08T05:45:51-07:00", - "end": "1988-06-08T05:45:51-07:00" - }, - "created": "1987-06-08T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:99f0bc94-4c7d-43b2-aa7d-a10111086f4c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1987-06-08T05:15:51-07:00", - "end": "1987-06-08T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f4e2c8ef-f3e6-4e80-9671-1f47ee1b7909" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413", - "resource": { - "resourceType": "Encounter", - "id": "676b4f93-6ea2-40f2-9c74-c8cd9f7ff413", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1987-08-10T05:15:51-07:00", - "end": "1987-08-10T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1987-08-10T05:15:51-07:00", - "end": "1987-08-10T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:941f3a51-4fce-4310-9cbc-543c82a65a1f", - "resource": { - "resourceType": "MedicationRequest", - "id": "941f3a51-4fce-4310-9cbc-543c82a65a1f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" - }, - "authoredOn": "1987-08-10T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:bc8e4b0c-0570-4ffc-b5f9-0a5f37174d3b", - "resource": { - "resourceType": "Claim", - "id": "bc8e4b0c-0570-4ffc-b5f9-0a5f37174d3b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-08-10T05:15:51-07:00", - "end": "1987-08-10T05:30:51-07:00" - }, - "created": "1987-08-10T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:941f3a51-4fce-4310-9cbc-543c82a65a1f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fbfa1ff7-f922-4174-866f-1b3c40ef041e", - "resource": { - "resourceType": "Claim", - "id": "fbfa1ff7-f922-4174-866f-1b3c40ef041e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1987-08-10T05:15:51-07:00", - "end": "1987-08-10T05:30:51-07:00" - }, - "created": "1987-08-10T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:aa63e71f-322c-4452-8c52-1f50fa3da177", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "aa63e71f-322c-4452-8c52-1f50fa3da177", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fbfa1ff7-f922-4174-866f-1b3c40ef041e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-08-10T05:30:51-07:00", - "end": "1988-08-10T05:30:51-07:00" - }, - "created": "1987-08-10T05:30:51-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:fbfa1ff7-f922-4174-866f-1b3c40ef041e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "1987-08-10T05:15:51-07:00", - "end": "1987-08-10T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:676b4f93-6ea2-40f2-9c74-c8cd9f7ff413" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e", - "resource": { - "resourceType": "Encounter", - "id": "0668f0dc-d937-4ddc-a810-ec5a8a0a541e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1987-08-05T05:15:51-07:00", - "end": "1987-08-05T05:42:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1987-08-05T05:15:51-07:00", - "end": "1987-08-05T05:42:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5ce622a3-32f9-442a-a25c-6fc4a4f5abc3", - "resource": { - "resourceType": "MedicationRequest", - "id": "5ce622a3-32f9-442a-a25c-6fc4a4f5abc3", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" - }, - "authoredOn": "1987-08-10T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:b187bdaf-b771-48e5-835d-99581ed7895a", - "resource": { - "resourceType": "Claim", - "id": "b187bdaf-b771-48e5-835d-99581ed7895a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-08-05T05:15:51-07:00", - "end": "1987-08-05T05:42:51-07:00" - }, - "created": "1987-08-05T05:42:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:5ce622a3-32f9-442a-a25c-6fc4a4f5abc3" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:78057408-8303-4cec-8355-962090104b54", - "resource": { - "resourceType": "Claim", - "id": "78057408-8303-4cec-8355-962090104b54", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1987-08-05T05:15:51-07:00", - "end": "1987-08-05T05:42:51-07:00" - }, - "created": "1987-08-05T05:42:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:103a5de2-f13a-4b30-b2d2-d509f9b0ce86", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "103a5de2-f13a-4b30-b2d2-d509f9b0ce86", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "78057408-8303-4cec-8355-962090104b54" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1987-08-05T05:42:51-07:00", - "end": "1988-08-05T05:42:51-07:00" - }, - "created": "1987-08-05T05:42:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:78057408-8303-4cec-8355-962090104b54" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "1987-08-05T05:15:51-07:00", - "end": "1987-08-05T05:42:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0668f0dc-d937-4ddc-a810-ec5a8a0a541e" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1", - "resource": { - "resourceType": "Encounter", - "id": "b60aad93-4868-4310-84ce-2c2a8b4885e1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1988-06-13T05:15:51-07:00", - "end": "1988-06-13T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1988-06-13T05:15:51-07:00", - "end": "1988-06-13T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bd1b16d1-af46-4223-9b6f-04dc012badc4", - "resource": { - "resourceType": "MedicationRequest", - "id": "bd1b16d1-af46-4223-9b6f-04dc012badc4", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" - }, - "authoredOn": "1988-06-13T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d50ae18a-e17c-439e-a5fa-9f81b3c7d396", - "resource": { - "resourceType": "Claim", - "id": "d50ae18a-e17c-439e-a5fa-9f81b3c7d396", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1988-06-13T05:15:51-07:00", - "end": "1988-06-13T05:30:51-07:00" - }, - "created": "1988-06-13T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:bd1b16d1-af46-4223-9b6f-04dc012badc4" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f4023f6d-78e8-451c-98a1-adfd07f2d8e5", - "resource": { - "resourceType": "Claim", - "id": "f4023f6d-78e8-451c-98a1-adfd07f2d8e5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1988-06-13T05:15:51-07:00", - "end": "1988-06-13T05:30:51-07:00" - }, - "created": "1988-06-13T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d8b0ab77-9b31-4d93-b60a-cc45da2b15ce", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d8b0ab77-9b31-4d93-b60a-cc45da2b15ce", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f4023f6d-78e8-451c-98a1-adfd07f2d8e5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1988-06-13T05:30:51-07:00", - "end": "1989-06-13T05:30:51-07:00" - }, - "created": "1988-06-13T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:f4023f6d-78e8-451c-98a1-adfd07f2d8e5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1988-06-13T05:15:51-07:00", - "end": "1988-06-13T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b60aad93-4868-4310-84ce-2c2a8b4885e1" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784", - "resource": { - "resourceType": "Encounter", - "id": "2e2dea08-86ab-4414-8776-4c1c2878c784", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1989-06-19T05:15:51-07:00", - "end": "1989-06-19T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1989-06-19T05:15:51-07:00", - "end": "1989-06-19T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fa5a1c27-248b-4f16-8abf-5b902ebd14b2", - "resource": { - "resourceType": "MedicationRequest", - "id": "fa5a1c27-248b-4f16-8abf-5b902ebd14b2", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" - }, - "authoredOn": "1989-06-19T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ed4dde50-a7e0-40e3-8acc-0ea0f3bf144c", - "resource": { - "resourceType": "Claim", - "id": "ed4dde50-a7e0-40e3-8acc-0ea0f3bf144c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1989-06-19T05:15:51-07:00", - "end": "1989-06-19T05:45:51-07:00" - }, - "created": "1989-06-19T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:fa5a1c27-248b-4f16-8abf-5b902ebd14b2" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9553833f-ca98-4126-8b52-ae921a23934e", - "resource": { - "resourceType": "Claim", - "id": "9553833f-ca98-4126-8b52-ae921a23934e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1989-06-19T05:15:51-07:00", - "end": "1989-06-19T05:45:51-07:00" - }, - "created": "1989-06-19T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:be6e8fc1-ab29-4dab-b93d-eebf62ccd9b5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "be6e8fc1-ab29-4dab-b93d-eebf62ccd9b5", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9553833f-ca98-4126-8b52-ae921a23934e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1989-06-19T05:45:51-07:00", - "end": "1990-06-19T05:45:51-07:00" - }, - "created": "1989-06-19T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9553833f-ca98-4126-8b52-ae921a23934e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1989-06-19T05:15:51-07:00", - "end": "1989-06-19T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2e2dea08-86ab-4414-8776-4c1c2878c784" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1", - "resource": { - "resourceType": "Encounter", - "id": "a60e4643-1275-4309-8d63-655275b334a1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1990-06-25T05:15:51-07:00", - "end": "1990-06-25T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1990-06-25T05:15:51-07:00", - "end": "1990-06-25T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1eff9cf5-43e3-4b36-9a92-43f50a5da3e6", - "resource": { - "resourceType": "MedicationRequest", - "id": "1eff9cf5-43e3-4b36-9a92-43f50a5da3e6", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" - }, - "authoredOn": "1990-06-25T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:b31486c8-2380-44f0-9d69-ab2f8cbf4784", - "resource": { - "resourceType": "Claim", - "id": "b31486c8-2380-44f0-9d69-ab2f8cbf4784", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1990-06-25T05:15:51-07:00", - "end": "1990-06-25T05:30:51-07:00" - }, - "created": "1990-06-25T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:1eff9cf5-43e3-4b36-9a92-43f50a5da3e6" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a24e410b-ff75-4db9-be01-d2b5cf9a6988", - "resource": { - "resourceType": "Claim", - "id": "a24e410b-ff75-4db9-be01-d2b5cf9a6988", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1990-06-25T05:15:51-07:00", - "end": "1990-06-25T05:30:51-07:00" - }, - "created": "1990-06-25T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0809360b-aeae-45f0-ae10-4487c9de961a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0809360b-aeae-45f0-ae10-4487c9de961a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a24e410b-ff75-4db9-be01-d2b5cf9a6988" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1990-06-25T05:30:51-07:00", - "end": "1991-06-25T05:30:51-07:00" - }, - "created": "1990-06-25T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a24e410b-ff75-4db9-be01-d2b5cf9a6988" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1990-06-25T05:15:51-07:00", - "end": "1990-06-25T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a60e4643-1275-4309-8d63-655275b334a1" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0", - "resource": { - "resourceType": "Encounter", - "id": "5c8d14a8-fb3b-472e-a1fd-065e3b7043f0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1991-06-15T05:15:51-07:00", - "end": "1991-06-15T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1991-06-15T05:15:51-07:00", - "end": "1991-06-15T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "75498004", - "display": "Acute bacterial sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c", - "resource": { - "resourceType": "Condition", - "id": "8a42cdb0-8952-4a22-b347-006f66bb1e0c", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "40055000", - "display": "Chronic sinusitis (disorder)" - } - ], - "text": "Chronic sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" - }, - "onsetDateTime": "1991-06-15T05:15:51-07:00", - "recordedDate": "1991-06-15T05:15:51-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ebb23fe1-d0b8-43df-bace-f2611ec93f1b", - "resource": { - "resourceType": "Claim", - "id": "ebb23fe1-d0b8-43df-bace-f2611ec93f1b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1991-06-15T05:15:51-07:00", - "end": "1991-06-15T05:30:51-07:00" - }, - "created": "1991-06-15T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "40055000", - "display": "Chronic sinusitis (disorder)" - } - ], - "text": "Chronic sinusitis (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2b5aade7-5c7f-41b7-9400-c5a3363f382f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2b5aade7-5c7f-41b7-9400-c5a3363f382f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ebb23fe1-d0b8-43df-bace-f2611ec93f1b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1991-06-15T05:30:51-07:00", - "end": "1992-06-15T05:30:51-07:00" - }, - "created": "1991-06-15T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ebb23fe1-d0b8-43df-bace-f2611ec93f1b" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8a42cdb0-8952-4a22-b347-006f66bb1e0c" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "1991-06-15T05:15:51-07:00", - "end": "1991-06-15T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:5c8d14a8-fb3b-472e-a1fd-065e3b7043f0" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "40055000", - "display": "Chronic sinusitis (disorder)" - } - ], - "text": "Chronic sinusitis (disorder)" - }, - "servicedPeriod": { - "start": "1991-06-15T05:15:51-07:00", - "end": "1991-06-15T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932", - "resource": { - "resourceType": "Encounter", - "id": "7ec7b383-ca2b-4d09-a1b1-c56615917932", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1991-06-24T05:15:51-07:00", - "end": "1991-06-24T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "1991-06-24T05:15:51-07:00", - "end": "1991-06-24T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:db639544-126f-4527-94cf-b5b84de4bea9", - "resource": { - "resourceType": "MedicationRequest", - "id": "db639544-126f-4527-94cf-b5b84de4bea9", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" - }, - "authoredOn": "1991-06-24T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1043af7a-95b3-4f63-b5ff-e85e61f29b98", - "resource": { - "resourceType": "Claim", - "id": "1043af7a-95b3-4f63-b5ff-e85e61f29b98", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1991-06-24T05:15:51-07:00", - "end": "1991-06-24T05:30:51-07:00" - }, - "created": "1991-06-24T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:db639544-126f-4527-94cf-b5b84de4bea9" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:10e409dd-2039-4b62-a408-25a8d5d7b245", - "resource": { - "resourceType": "Claim", - "id": "10e409dd-2039-4b62-a408-25a8d5d7b245", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1991-06-24T05:15:51-07:00", - "end": "1991-06-24T05:30:51-07:00" - }, - "created": "1991-06-24T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ce9fa012-bad9-409b-a82f-904b921046f3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ce9fa012-bad9-409b-a82f-904b921046f3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "10e409dd-2039-4b62-a408-25a8d5d7b245" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1991-06-24T05:30:51-07:00", - "end": "1992-06-24T05:30:51-07:00" - }, - "created": "1991-06-24T05:30:51-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:10e409dd-2039-4b62-a408-25a8d5d7b245" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "1991-06-24T05:15:51-07:00", - "end": "1991-06-24T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7ec7b383-ca2b-4d09-a1b1-c56615917932" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350", - "resource": { - "resourceType": "Encounter", - "id": "45a4ff1f-b1ae-4d3a-8c17-4a162967a350", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1991-07-01T05:15:51-07:00", - "end": "1991-07-01T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1991-07-01T05:15:51-07:00", - "end": "1991-07-01T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:62f16944-78e7-49e6-8cba-a1063c5567e7", - "resource": { - "resourceType": "MedicationRequest", - "id": "62f16944-78e7-49e6-8cba-a1063c5567e7", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" - }, - "authoredOn": "1991-07-01T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:af037f85-2bb8-40e5-80d5-20ae936d61db", - "resource": { - "resourceType": "Claim", - "id": "af037f85-2bb8-40e5-80d5-20ae936d61db", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1991-07-01T05:15:51-07:00", - "end": "1991-07-01T05:45:51-07:00" - }, - "created": "1991-07-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:62f16944-78e7-49e6-8cba-a1063c5567e7" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b2100338-aadd-49d8-8f7b-6495fc062d0a", - "resource": { - "resourceType": "Claim", - "id": "b2100338-aadd-49d8-8f7b-6495fc062d0a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1991-07-01T05:15:51-07:00", - "end": "1991-07-01T05:45:51-07:00" - }, - "created": "1991-07-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6e35c6f1-a888-4d35-8eb0-3d477b257766", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6e35c6f1-a888-4d35-8eb0-3d477b257766", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b2100338-aadd-49d8-8f7b-6495fc062d0a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1991-07-01T05:45:51-07:00", - "end": "1992-07-01T05:45:51-07:00" - }, - "created": "1991-07-01T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b2100338-aadd-49d8-8f7b-6495fc062d0a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1991-07-01T05:15:51-07:00", - "end": "1991-07-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:45a4ff1f-b1ae-4d3a-8c17-4a162967a350" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be", - "resource": { - "resourceType": "Encounter", - "id": "30e7ddad-820e-4e39-b748-8426db7346be", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1992-07-06T05:15:51-07:00", - "end": "1992-07-06T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1992-07-06T05:15:51-07:00", - "end": "1992-07-06T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ed645b29-f27d-4b5a-8f75-88539e2760da", - "resource": { - "resourceType": "MedicationRequest", - "id": "ed645b29-f27d-4b5a-8f75-88539e2760da", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" - }, - "authoredOn": "1992-07-06T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:deb8feb2-58b0-43c3-9da1-653ad057373a", - "resource": { - "resourceType": "Claim", - "id": "deb8feb2-58b0-43c3-9da1-653ad057373a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1992-07-06T05:15:51-07:00", - "end": "1992-07-06T05:45:51-07:00" - }, - "created": "1992-07-06T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ed645b29-f27d-4b5a-8f75-88539e2760da" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8dde5a5f-cf94-4bb6-9ace-f635f1a977f1", - "resource": { - "resourceType": "Claim", - "id": "8dde5a5f-cf94-4bb6-9ace-f635f1a977f1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1992-07-06T05:15:51-07:00", - "end": "1992-07-06T05:45:51-07:00" - }, - "created": "1992-07-06T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:47681d39-1c4e-4cc1-9617-08cc31fd97b1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "47681d39-1c4e-4cc1-9617-08cc31fd97b1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8dde5a5f-cf94-4bb6-9ace-f635f1a977f1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1992-07-06T05:45:51-07:00", - "end": "1993-07-06T05:45:51-07:00" - }, - "created": "1992-07-06T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8dde5a5f-cf94-4bb6-9ace-f635f1a977f1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1992-07-06T05:15:51-07:00", - "end": "1992-07-06T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:30e7ddad-820e-4e39-b748-8426db7346be" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d", - "resource": { - "resourceType": "Encounter", - "id": "0250c11d-b2e5-4707-8833-3ad4e2946f4d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1993-07-12T05:15:51-07:00", - "end": "1993-07-12T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1993-07-12T05:15:51-07:00", - "end": "1993-07-12T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b1add7db-1732-4fba-8b8c-ea854269c77e", - "resource": { - "resourceType": "MedicationRequest", - "id": "b1add7db-1732-4fba-8b8c-ea854269c77e", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" - }, - "authoredOn": "1993-07-12T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8acc7c2f-dcce-47c1-84cb-3eefe96a1a67", - "resource": { - "resourceType": "Claim", - "id": "8acc7c2f-dcce-47c1-84cb-3eefe96a1a67", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1993-07-12T05:15:51-07:00", - "end": "1993-07-12T05:30:51-07:00" - }, - "created": "1993-07-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:b1add7db-1732-4fba-8b8c-ea854269c77e" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:55458916-e5bb-43f9-9893-f613888efc51", - "resource": { - "resourceType": "Claim", - "id": "55458916-e5bb-43f9-9893-f613888efc51", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1993-07-12T05:15:51-07:00", - "end": "1993-07-12T05:30:51-07:00" - }, - "created": "1993-07-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cded643f-47f5-40b7-ab8f-931c3af87a12", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cded643f-47f5-40b7-ab8f-931c3af87a12", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "55458916-e5bb-43f9-9893-f613888efc51" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1993-07-12T05:30:51-07:00", - "end": "1994-07-12T05:30:51-07:00" - }, - "created": "1993-07-12T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:55458916-e5bb-43f9-9893-f613888efc51" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1993-07-12T05:15:51-07:00", - "end": "1993-07-12T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0250c11d-b2e5-4707-8833-3ad4e2946f4d" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0", - "resource": { - "resourceType": "Encounter", - "id": "64a03bca-7523-444a-8315-7d6adddb1bc0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1994-07-18T05:15:51-07:00", - "end": "1994-07-18T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1994-07-18T05:15:51-07:00", - "end": "1994-07-18T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bee3c907-561b-4cf2-8678-0fab5469d755", - "resource": { - "resourceType": "MedicationRequest", - "id": "bee3c907-561b-4cf2-8678-0fab5469d755", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" - }, - "authoredOn": "1994-07-18T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:25496d8c-3e18-4176-9280-450d0b12e22e", - "resource": { - "resourceType": "Claim", - "id": "25496d8c-3e18-4176-9280-450d0b12e22e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1994-07-18T05:15:51-07:00", - "end": "1994-07-18T05:30:51-07:00" - }, - "created": "1994-07-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:bee3c907-561b-4cf2-8678-0fab5469d755" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:017eb07c-04ed-461d-8646-afdfa3f2c22c", - "resource": { - "resourceType": "Claim", - "id": "017eb07c-04ed-461d-8646-afdfa3f2c22c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1994-07-18T05:15:51-07:00", - "end": "1994-07-18T05:30:51-07:00" - }, - "created": "1994-07-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f2d5594d-89d8-4758-8403-4d29a157da54", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f2d5594d-89d8-4758-8403-4d29a157da54", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "017eb07c-04ed-461d-8646-afdfa3f2c22c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1994-07-18T05:30:51-07:00", - "end": "1995-07-18T05:30:51-07:00" - }, - "created": "1994-07-18T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:017eb07c-04ed-461d-8646-afdfa3f2c22c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1994-07-18T05:15:51-07:00", - "end": "1994-07-18T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:64a03bca-7523-444a-8315-7d6adddb1bc0" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca", - "resource": { - "resourceType": "Encounter", - "id": "a663c449-b5ad-4bc9-b667-117b421f0eca", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1995-07-24T05:15:51-07:00", - "end": "1995-07-24T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1995-07-24T05:15:51-07:00", - "end": "1995-07-24T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6e8d335c-6fe5-4823-9aad-8d7ec0718ea3", - "resource": { - "resourceType": "MedicationRequest", - "id": "6e8d335c-6fe5-4823-9aad-8d7ec0718ea3", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" - }, - "authoredOn": "1995-07-24T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:80c4765c-75fd-44ee-a345-6d802a601eb8", - "resource": { - "resourceType": "Claim", - "id": "80c4765c-75fd-44ee-a345-6d802a601eb8", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1995-07-24T05:15:51-07:00", - "end": "1995-07-24T05:45:51-07:00" - }, - "created": "1995-07-24T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:6e8d335c-6fe5-4823-9aad-8d7ec0718ea3" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2d36dc4b-f1d9-49b4-8ccc-884717e20688", - "resource": { - "resourceType": "Claim", - "id": "2d36dc4b-f1d9-49b4-8ccc-884717e20688", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1995-07-24T05:15:51-07:00", - "end": "1995-07-24T05:45:51-07:00" - }, - "created": "1995-07-24T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:84810259-ab17-4b27-8464-ed2ad8a2bc83", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "84810259-ab17-4b27-8464-ed2ad8a2bc83", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2d36dc4b-f1d9-49b4-8ccc-884717e20688" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1995-07-24T05:45:51-07:00", - "end": "1996-07-24T05:45:51-07:00" - }, - "created": "1995-07-24T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2d36dc4b-f1d9-49b4-8ccc-884717e20688" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1995-07-24T05:15:51-07:00", - "end": "1995-07-24T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a663c449-b5ad-4bc9-b667-117b421f0eca" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f", - "resource": { - "resourceType": "Encounter", - "id": "7c1cbc44-fed4-46c1-9ea8-236b1119153f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1996-07-29T05:15:51-07:00", - "end": "1996-07-29T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1996-07-29T05:15:51-07:00", - "end": "1996-07-29T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:67bcca74-181d-43e4-bd88-8528123ef198", - "resource": { - "resourceType": "MedicationRequest", - "id": "67bcca74-181d-43e4-bd88-8528123ef198", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" - }, - "authoredOn": "1996-07-29T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:634fd458-9041-47f8-a9a0-2bb905940f2e", - "resource": { - "resourceType": "Claim", - "id": "634fd458-9041-47f8-a9a0-2bb905940f2e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1996-07-29T05:15:51-07:00", - "end": "1996-07-29T05:45:51-07:00" - }, - "created": "1996-07-29T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:67bcca74-181d-43e4-bd88-8528123ef198" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:073d30d3-ad05-492c-9dc2-09e878dd1b04", - "resource": { - "resourceType": "Claim", - "id": "073d30d3-ad05-492c-9dc2-09e878dd1b04", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1996-07-29T05:15:51-07:00", - "end": "1996-07-29T05:45:51-07:00" - }, - "created": "1996-07-29T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:31f1002f-2e6c-4367-8736-8685cd95d461", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "31f1002f-2e6c-4367-8736-8685cd95d461", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "073d30d3-ad05-492c-9dc2-09e878dd1b04" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1996-07-29T05:45:51-07:00", - "end": "1997-07-29T05:45:51-07:00" - }, - "created": "1996-07-29T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:073d30d3-ad05-492c-9dc2-09e878dd1b04" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1996-07-29T05:15:51-07:00", - "end": "1996-07-29T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7c1cbc44-fed4-46c1-9ea8-236b1119153f" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452", - "resource": { - "resourceType": "Encounter", - "id": "639840e0-52f5-4aae-9cc1-c35cbb30d452", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1997-08-04T05:15:51-07:00", - "end": "1997-08-04T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1997-08-04T05:15:51-07:00", - "end": "1997-08-04T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:79ea2871-e47b-4778-8035-c3c21c457976", - "resource": { - "resourceType": "MedicationRequest", - "id": "79ea2871-e47b-4778-8035-c3c21c457976", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" - }, - "authoredOn": "1997-08-04T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0db36527-224d-484f-85e4-0701c4939844", - "resource": { - "resourceType": "Claim", - "id": "0db36527-224d-484f-85e4-0701c4939844", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1997-08-04T05:15:51-07:00", - "end": "1997-08-04T05:45:51-07:00" - }, - "created": "1997-08-04T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:79ea2871-e47b-4778-8035-c3c21c457976" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9b293966-61cf-459b-b77e-40340c1a1f28", - "resource": { - "resourceType": "Claim", - "id": "9b293966-61cf-459b-b77e-40340c1a1f28", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1997-08-04T05:15:51-07:00", - "end": "1997-08-04T05:45:51-07:00" - }, - "created": "1997-08-04T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7d49fe77-98c8-4ba2-bfd8-9feb662bdb1c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7d49fe77-98c8-4ba2-bfd8-9feb662bdb1c", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9b293966-61cf-459b-b77e-40340c1a1f28" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1997-08-04T05:45:51-07:00", - "end": "1998-08-04T05:45:51-07:00" - }, - "created": "1997-08-04T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9b293966-61cf-459b-b77e-40340c1a1f28" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1997-08-04T05:15:51-07:00", - "end": "1997-08-04T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:639840e0-52f5-4aae-9cc1-c35cbb30d452" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740", - "resource": { - "resourceType": "Encounter", - "id": "a8466f85-3a1a-4247-9e88-80e3f9912740", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1998-08-10T05:15:51-07:00", - "end": "1998-08-10T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1998-08-10T05:15:51-07:00", - "end": "1998-08-10T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ae6181f9-10bd-4ab4-8796-98de69c543da", - "resource": { - "resourceType": "MedicationRequest", - "id": "ae6181f9-10bd-4ab4-8796-98de69c543da", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" - }, - "authoredOn": "1998-08-10T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d69ac9d4-885b-4205-8ffb-aac4bdf01241", - "resource": { - "resourceType": "Claim", - "id": "d69ac9d4-885b-4205-8ffb-aac4bdf01241", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1998-08-10T05:15:51-07:00", - "end": "1998-08-10T05:45:51-07:00" - }, - "created": "1998-08-10T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ae6181f9-10bd-4ab4-8796-98de69c543da" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ca66cb47-3921-4d9f-ac0b-b2f4473f27cb", - "resource": { - "resourceType": "Claim", - "id": "ca66cb47-3921-4d9f-ac0b-b2f4473f27cb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1998-08-10T05:15:51-07:00", - "end": "1998-08-10T05:45:51-07:00" - }, - "created": "1998-08-10T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f47609a2-10b5-496f-a0a8-0a9ccf3729a8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f47609a2-10b5-496f-a0a8-0a9ccf3729a8", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ca66cb47-3921-4d9f-ac0b-b2f4473f27cb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1998-08-10T05:45:51-07:00", - "end": "1999-08-10T05:45:51-07:00" - }, - "created": "1998-08-10T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ca66cb47-3921-4d9f-ac0b-b2f4473f27cb" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1998-08-10T05:15:51-07:00", - "end": "1998-08-10T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a8466f85-3a1a-4247-9e88-80e3f9912740" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5", - "resource": { - "resourceType": "Encounter", - "id": "6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1999-08-16T05:15:51-07:00", - "end": "1999-08-16T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "1999-08-16T05:15:51-07:00", - "end": "1999-08-16T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0cd60bac-b1dc-4dd6-9328-d3cf98976665", - "resource": { - "resourceType": "MedicationRequest", - "id": "0cd60bac-b1dc-4dd6-9328-d3cf98976665", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" - }, - "authoredOn": "1999-08-16T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d062ac5f-6f95-4212-9936-cde99cc84342", - "resource": { - "resourceType": "Claim", - "id": "d062ac5f-6f95-4212-9936-cde99cc84342", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1999-08-16T05:15:51-07:00", - "end": "1999-08-16T05:30:51-07:00" - }, - "created": "1999-08-16T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0cd60bac-b1dc-4dd6-9328-d3cf98976665" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:32e036b7-6ade-46d7-93a7-aa83cbf7811c", - "resource": { - "resourceType": "Claim", - "id": "32e036b7-6ade-46d7-93a7-aa83cbf7811c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "1999-08-16T05:15:51-07:00", - "end": "1999-08-16T05:30:51-07:00" - }, - "created": "1999-08-16T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:024a2154-971d-4861-b4c1-9ac5f54479d0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "024a2154-971d-4861-b4c1-9ac5f54479d0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "32e036b7-6ade-46d7-93a7-aa83cbf7811c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "1999-08-16T05:30:51-07:00", - "end": "2000-08-16T05:30:51-07:00" - }, - "created": "1999-08-16T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:32e036b7-6ade-46d7-93a7-aa83cbf7811c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "1999-08-16T05:15:51-07:00", - "end": "1999-08-16T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:6f3f4e14-0f6b-4c5a-95a9-4b8818608ab5" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c", - "resource": { - "resourceType": "Encounter", - "id": "15cb804f-05bf-4507-a470-fff28bc7f82c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2000-08-21T05:15:51-07:00", - "end": "2000-08-21T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2000-08-21T05:15:51-07:00", - "end": "2000-08-21T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:491b8f1e-db27-4025-9a5a-cd6a4d39de43", - "resource": { - "resourceType": "MedicationRequest", - "id": "491b8f1e-db27-4025-9a5a-cd6a4d39de43", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" - }, - "authoredOn": "2000-08-21T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:6fa61582-e619-4e62-948d-2d81de09332d", - "resource": { - "resourceType": "Claim", - "id": "6fa61582-e619-4e62-948d-2d81de09332d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2000-08-21T05:15:51-07:00", - "end": "2000-08-21T05:30:51-07:00" - }, - "created": "2000-08-21T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:491b8f1e-db27-4025-9a5a-cd6a4d39de43" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a2f13fd6-c438-41e6-bab8-6b1a4663f454", - "resource": { - "resourceType": "Claim", - "id": "a2f13fd6-c438-41e6-bab8-6b1a4663f454", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2000-08-21T05:15:51-07:00", - "end": "2000-08-21T05:30:51-07:00" - }, - "created": "2000-08-21T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:27507818-5e13-4058-91b6-e12fd9e7a556", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "27507818-5e13-4058-91b6-e12fd9e7a556", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a2f13fd6-c438-41e6-bab8-6b1a4663f454" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2000-08-21T05:30:51-07:00", - "end": "2001-08-21T05:30:51-07:00" - }, - "created": "2000-08-21T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a2f13fd6-c438-41e6-bab8-6b1a4663f454" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2000-08-21T05:15:51-07:00", - "end": "2000-08-21T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:15cb804f-05bf-4507-a470-fff28bc7f82c" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b", - "resource": { - "resourceType": "Encounter", - "id": "e3bd9072-a533-48e6-ad52-8d79f0358f6b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2001-08-27T05:15:51-07:00", - "end": "2001-08-27T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2001-08-27T05:15:51-07:00", - "end": "2001-08-27T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f3d66454-5fa5-4f72-80da-eed86bfa1704", - "resource": { - "resourceType": "MedicationRequest", - "id": "f3d66454-5fa5-4f72-80da-eed86bfa1704", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" - }, - "authoredOn": "2001-08-27T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:65451aa1-cd45-4c73-a969-04e58d815bf9", - "resource": { - "resourceType": "Claim", - "id": "65451aa1-cd45-4c73-a969-04e58d815bf9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2001-08-27T05:15:51-07:00", - "end": "2001-08-27T05:30:51-07:00" - }, - "created": "2001-08-27T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:f3d66454-5fa5-4f72-80da-eed86bfa1704" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2626ec55-4590-4216-afa1-9b2e7848553a", - "resource": { - "resourceType": "Claim", - "id": "2626ec55-4590-4216-afa1-9b2e7848553a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2001-08-27T05:15:51-07:00", - "end": "2001-08-27T05:30:51-07:00" - }, - "created": "2001-08-27T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b6f36a29-c84a-467a-8395-934d9ab358d4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b6f36a29-c84a-467a-8395-934d9ab358d4", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2626ec55-4590-4216-afa1-9b2e7848553a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2001-08-27T05:30:51-07:00", - "end": "2002-08-27T05:30:51-07:00" - }, - "created": "2001-08-27T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2626ec55-4590-4216-afa1-9b2e7848553a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2001-08-27T05:15:51-07:00", - "end": "2001-08-27T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e3bd9072-a533-48e6-ad52-8d79f0358f6b" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393", - "resource": { - "resourceType": "Encounter", - "id": "69d8df57-0ae8-442c-96ef-9bf824f0c393", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2002-09-02T05:15:51-07:00", - "end": "2002-09-02T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2002-09-02T05:15:51-07:00", - "end": "2002-09-02T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ccf20089-8271-4f70-9b81-4b8db495f873", - "resource": { - "resourceType": "MedicationRequest", - "id": "ccf20089-8271-4f70-9b81-4b8db495f873", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" - }, - "authoredOn": "2002-09-02T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:c63cf98c-45da-4708-940a-2ff8d0820ba9", - "resource": { - "resourceType": "Claim", - "id": "c63cf98c-45da-4708-940a-2ff8d0820ba9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2002-09-02T05:15:51-07:00", - "end": "2002-09-02T05:30:51-07:00" - }, - "created": "2002-09-02T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ccf20089-8271-4f70-9b81-4b8db495f873" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ed2a44a0-d34a-4d1a-90fb-98ee417f8686", - "resource": { - "resourceType": "Claim", - "id": "ed2a44a0-d34a-4d1a-90fb-98ee417f8686", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2002-09-02T05:15:51-07:00", - "end": "2002-09-02T05:30:51-07:00" - }, - "created": "2002-09-02T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:43482409-317b-4c09-b465-fbc2afba7d03", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "43482409-317b-4c09-b465-fbc2afba7d03", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ed2a44a0-d34a-4d1a-90fb-98ee417f8686" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2002-09-02T05:30:51-07:00", - "end": "2003-09-02T05:30:51-07:00" - }, - "created": "2002-09-02T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ed2a44a0-d34a-4d1a-90fb-98ee417f8686" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2002-09-02T05:15:51-07:00", - "end": "2002-09-02T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:69d8df57-0ae8-442c-96ef-9bf824f0c393" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001", - "resource": { - "resourceType": "Encounter", - "id": "38734c49-0953-4f17-80fe-f16483bdf001", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2003-09-08T05:15:51-07:00", - "end": "2003-09-08T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2003-09-08T05:15:51-07:00", - "end": "2003-09-08T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7c36c44e-166b-4c3e-a473-b88793457aee", - "resource": { - "resourceType": "MedicationRequest", - "id": "7c36c44e-166b-4c3e-a473-b88793457aee", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" - }, - "authoredOn": "2003-09-08T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:4f090bfb-4409-433d-8546-c26faa0808f3", - "resource": { - "resourceType": "Claim", - "id": "4f090bfb-4409-433d-8546-c26faa0808f3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2003-09-08T05:15:51-07:00", - "end": "2003-09-08T05:30:51-07:00" - }, - "created": "2003-09-08T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:7c36c44e-166b-4c3e-a473-b88793457aee" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e5e73e04-4a53-4d57-8084-5152403ac862", - "resource": { - "resourceType": "Claim", - "id": "e5e73e04-4a53-4d57-8084-5152403ac862", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2003-09-08T05:15:51-07:00", - "end": "2003-09-08T05:30:51-07:00" - }, - "created": "2003-09-08T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5d39a1e4-f46b-4fa1-9e5b-f3ec8e827f43", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5d39a1e4-f46b-4fa1-9e5b-f3ec8e827f43", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e5e73e04-4a53-4d57-8084-5152403ac862" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2003-09-08T05:30:51-07:00", - "end": "2004-09-08T05:30:51-07:00" - }, - "created": "2003-09-08T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e5e73e04-4a53-4d57-8084-5152403ac862" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2003-09-08T05:15:51-07:00", - "end": "2003-09-08T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:38734c49-0953-4f17-80fe-f16483bdf001" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873", - "resource": { - "resourceType": "Encounter", - "id": "3c5c554f-f12a-4522-8ecd-9dd06f1c9873", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2004-09-13T05:15:51-07:00", - "end": "2004-09-13T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2004-09-13T05:15:51-07:00", - "end": "2004-09-13T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0e339852-87ed-4037-a76b-243b3ae4866d", - "resource": { - "resourceType": "MedicationRequest", - "id": "0e339852-87ed-4037-a76b-243b3ae4866d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" - }, - "authoredOn": "2004-09-13T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:fb07bf2a-0d66-46e1-be9a-4c652675e7e3", - "resource": { - "resourceType": "Claim", - "id": "fb07bf2a-0d66-46e1-be9a-4c652675e7e3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2004-09-13T05:15:51-07:00", - "end": "2004-09-13T05:45:51-07:00" - }, - "created": "2004-09-13T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:0e339852-87ed-4037-a76b-243b3ae4866d" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:37306189-78d5-43b4-a46a-e6b3fe59145e", - "resource": { - "resourceType": "Claim", - "id": "37306189-78d5-43b4-a46a-e6b3fe59145e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2004-09-13T05:15:51-07:00", - "end": "2004-09-13T05:45:51-07:00" - }, - "created": "2004-09-13T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:31684e94-d2dd-4534-ba52-308a2f150454", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "31684e94-d2dd-4534-ba52-308a2f150454", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "37306189-78d5-43b4-a46a-e6b3fe59145e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2004-09-13T05:45:51-07:00", - "end": "2005-09-13T05:45:51-07:00" - }, - "created": "2004-09-13T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:37306189-78d5-43b4-a46a-e6b3fe59145e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2004-09-13T05:15:51-07:00", - "end": "2004-09-13T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3c5c554f-f12a-4522-8ecd-9dd06f1c9873" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a", - "resource": { - "resourceType": "Encounter", - "id": "bde5f2d1-06f2-4ca1-8ca5-edf99867f02a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2005-09-19T05:15:51-07:00", - "end": "2005-09-19T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2005-09-19T05:15:51-07:00", - "end": "2005-09-19T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9ffc549f-07e9-4518-a20c-c39ea93316a9", - "resource": { - "resourceType": "MedicationRequest", - "id": "9ffc549f-07e9-4518-a20c-c39ea93316a9", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" - }, - "authoredOn": "2005-09-19T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:fcbd6b63-2bc3-4784-95c5-d0128f3b0509", - "resource": { - "resourceType": "Claim", - "id": "fcbd6b63-2bc3-4784-95c5-d0128f3b0509", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2005-09-19T05:15:51-07:00", - "end": "2005-09-19T05:45:51-07:00" - }, - "created": "2005-09-19T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:9ffc549f-07e9-4518-a20c-c39ea93316a9" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2f1294ad-5105-4447-aaa1-5277ee65be84", - "resource": { - "resourceType": "Claim", - "id": "2f1294ad-5105-4447-aaa1-5277ee65be84", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2005-09-19T05:15:51-07:00", - "end": "2005-09-19T05:45:51-07:00" - }, - "created": "2005-09-19T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:40dd5f2c-5f0d-4f7e-8cbd-af506b784ef0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "40dd5f2c-5f0d-4f7e-8cbd-af506b784ef0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2f1294ad-5105-4447-aaa1-5277ee65be84" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2005-09-19T05:45:51-07:00", - "end": "2006-09-19T05:45:51-07:00" - }, - "created": "2005-09-19T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2f1294ad-5105-4447-aaa1-5277ee65be84" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2005-09-19T05:15:51-07:00", - "end": "2005-09-19T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:bde5f2d1-06f2-4ca1-8ca5-edf99867f02a" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029", - "resource": { - "resourceType": "Encounter", - "id": "c7f5266a-1840-4fdc-bac1-39e12ab7d029", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2006-09-25T05:15:51-07:00", - "end": "2006-09-25T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2006-09-25T05:15:51-07:00", - "end": "2006-09-25T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c3e08e44-0734-4aa5-88d1-97b70e297e82", - "resource": { - "resourceType": "MedicationRequest", - "id": "c3e08e44-0734-4aa5-88d1-97b70e297e82", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" - }, - "authoredOn": "2006-09-25T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0e90715b-a68e-4cba-8097-7e02bde5ed0f", - "resource": { - "resourceType": "Claim", - "id": "0e90715b-a68e-4cba-8097-7e02bde5ed0f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2006-09-25T05:15:51-07:00", - "end": "2006-09-25T05:45:51-07:00" - }, - "created": "2006-09-25T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:c3e08e44-0734-4aa5-88d1-97b70e297e82" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:578cf805-ce3e-4b7e-879b-20fe1e733393", - "resource": { - "resourceType": "Claim", - "id": "578cf805-ce3e-4b7e-879b-20fe1e733393", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2006-09-25T05:15:51-07:00", - "end": "2006-09-25T05:45:51-07:00" - }, - "created": "2006-09-25T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7a45b794-7021-4b6a-bb27-becbbc7926c1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7a45b794-7021-4b6a-bb27-becbbc7926c1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "578cf805-ce3e-4b7e-879b-20fe1e733393" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2006-09-25T05:45:51-07:00", - "end": "2007-09-25T05:45:51-07:00" - }, - "created": "2006-09-25T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:578cf805-ce3e-4b7e-879b-20fe1e733393" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2006-09-25T05:15:51-07:00", - "end": "2006-09-25T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c7f5266a-1840-4fdc-bac1-39e12ab7d029" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408", - "resource": { - "resourceType": "Encounter", - "id": "8da5a258-0811-482e-bf4f-ae41d8c9b408", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2007-10-01T05:15:51-07:00", - "end": "2007-10-01T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2007-10-01T05:15:51-07:00", - "end": "2007-10-01T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f42f87c9-776a-4e5b-8866-9b5a1c1ca873", - "resource": { - "resourceType": "MedicationRequest", - "id": "f42f87c9-776a-4e5b-8866-9b5a1c1ca873", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" - }, - "authoredOn": "2007-10-01T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5c6b001e-c236-4960-8753-d2ba6bd6810d", - "resource": { - "resourceType": "Claim", - "id": "5c6b001e-c236-4960-8753-d2ba6bd6810d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2007-10-01T05:15:51-07:00", - "end": "2007-10-01T05:45:51-07:00" - }, - "created": "2007-10-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:f42f87c9-776a-4e5b-8866-9b5a1c1ca873" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cc07e348-c328-45b7-b77d-5a84aee092a8", - "resource": { - "resourceType": "Claim", - "id": "cc07e348-c328-45b7-b77d-5a84aee092a8", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2007-10-01T05:15:51-07:00", - "end": "2007-10-01T05:45:51-07:00" - }, - "created": "2007-10-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7b74363c-446b-4a0a-9400-05e58af6f205", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7b74363c-446b-4a0a-9400-05e58af6f205", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cc07e348-c328-45b7-b77d-5a84aee092a8" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2007-10-01T05:45:51-07:00", - "end": "2008-10-01T05:45:51-07:00" - }, - "created": "2007-10-01T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:cc07e348-c328-45b7-b77d-5a84aee092a8" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2007-10-01T05:15:51-07:00", - "end": "2007-10-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:8da5a258-0811-482e-bf4f-ae41d8c9b408" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66", - "resource": { - "resourceType": "Encounter", - "id": "d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2008-10-06T05:15:51-07:00", - "end": "2008-10-06T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2008-10-06T05:15:51-07:00", - "end": "2008-10-06T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:569719e7-88cc-4b94-b50d-0f1b09c3a241", - "resource": { - "resourceType": "MedicationRequest", - "id": "569719e7-88cc-4b94-b50d-0f1b09c3a241", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" - }, - "authoredOn": "2008-10-06T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:12ad2407-fc50-4bb1-bfb0-13f3c94a47c6", - "resource": { - "resourceType": "Claim", - "id": "12ad2407-fc50-4bb1-bfb0-13f3c94a47c6", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2008-10-06T05:15:51-07:00", - "end": "2008-10-06T05:30:51-07:00" - }, - "created": "2008-10-06T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:569719e7-88cc-4b94-b50d-0f1b09c3a241" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5e7c6df6-04f6-43ac-bcd3-3eda573322eb", - "resource": { - "resourceType": "Claim", - "id": "5e7c6df6-04f6-43ac-bcd3-3eda573322eb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2008-10-06T05:15:51-07:00", - "end": "2008-10-06T05:30:51-07:00" - }, - "created": "2008-10-06T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ca3c618e-d5bd-42f3-a387-bff54ef7ccef", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ca3c618e-d5bd-42f3-a387-bff54ef7ccef", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5e7c6df6-04f6-43ac-bcd3-3eda573322eb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2008-10-06T05:30:51-07:00", - "end": "2009-10-06T05:30:51-07:00" - }, - "created": "2008-10-06T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5e7c6df6-04f6-43ac-bcd3-3eda573322eb" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2008-10-06T05:15:51-07:00", - "end": "2008-10-06T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:d6fd12da-dd3d-40c3-9a77-e7e18c4b2c66" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb", - "resource": { - "resourceType": "Encounter", - "id": "e64717ac-fe72-473a-a97d-ca260c8e79fb", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2009-10-12T05:15:51-07:00", - "end": "2009-10-12T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2009-10-12T05:15:51-07:00", - "end": "2009-10-12T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:735e3ecf-e9e6-4fc0-a560-73c7764aab82", - "resource": { - "resourceType": "MedicationRequest", - "id": "735e3ecf-e9e6-4fc0-a560-73c7764aab82", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" - }, - "authoredOn": "2009-10-12T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:21eb7a78-ab08-4f40-a6bb-5d6622661f42", - "resource": { - "resourceType": "Claim", - "id": "21eb7a78-ab08-4f40-a6bb-5d6622661f42", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2009-10-12T05:15:51-07:00", - "end": "2009-10-12T05:30:51-07:00" - }, - "created": "2009-10-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:735e3ecf-e9e6-4fc0-a560-73c7764aab82" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8b50a202-84e3-4d11-8d29-f722a213882b", - "resource": { - "resourceType": "Claim", - "id": "8b50a202-84e3-4d11-8d29-f722a213882b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2009-10-12T05:15:51-07:00", - "end": "2009-10-12T05:30:51-07:00" - }, - "created": "2009-10-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b85e16fc-b6f6-4a20-9489-0fd0eb1ee9d6", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b85e16fc-b6f6-4a20-9489-0fd0eb1ee9d6", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8b50a202-84e3-4d11-8d29-f722a213882b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2009-10-12T05:30:51-07:00", - "end": "2010-10-12T05:30:51-07:00" - }, - "created": "2009-10-12T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8b50a202-84e3-4d11-8d29-f722a213882b" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2009-10-12T05:15:51-07:00", - "end": "2009-10-12T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e64717ac-fe72-473a-a97d-ca260c8e79fb" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6", - "resource": { - "resourceType": "Encounter", - "id": "1222f480-de35-4581-8234-aac3e0aaebf6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:140de0d2-40c2-44ca-9f44-6f25bb639b21", - "resource": { - "resourceType": "Observation", - "id": "140de0d2-40c2-44ca-9f44-6f25bb639b21", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47c76101-3c84-4587-884e-1b15fa4ed4ef", - "resource": { - "resourceType": "Observation", - "id": "47c76101-3c84-4587-884e-1b15fa4ed4ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:edecffeb-3509-47ba-a1f1-b7ab7ee1e101", - "resource": { - "resourceType": "Observation", - "id": "edecffeb-3509-47ba-a1f1-b7ab7ee1e101", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4172e8e9-596f-4431-8238-15bbd2d545ae", - "resource": { - "resourceType": "Observation", - "id": "4172e8e9-596f-4431-8238-15bbd2d545ae", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f350d116-2f8e-484d-b5a2-ab30d6dc0153", - "resource": { - "resourceType": "Observation", - "id": "f350d116-2f8e-484d-b5a2-ab30d6dc0153", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 106, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36ea7764-6c28-492a-83ce-82d2ef9e882c", - "resource": { - "resourceType": "Observation", - "id": "36ea7764-6c28-492a-83ce-82d2ef9e882c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 79, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb9276b8-1438-4be4-8f91-5cab50a69f70", - "resource": { - "resourceType": "Observation", - "id": "cb9276b8-1438-4be4-8f91-5cab50a69f70", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e0ef354-df58-4853-b411-d4f5a8f89076", - "resource": { - "resourceType": "Observation", - "id": "4e0ef354-df58-4853-b411-d4f5a8f89076", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 70.01, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7866c71e-5535-4847-b40a-e612c3a7bb98", - "resource": { - "resourceType": "Observation", - "id": "7866c71e-5535-4847-b40a-e612c3a7bb98", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 14.38, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6c369d1-c352-489f-8c55-8d5bcb4c59c7", - "resource": { - "resourceType": "Observation", - "id": "b6c369d1-c352-489f-8c55-8d5bcb4c59c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.74, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fabfb47f-c5e3-42ec-9c96-0e3760a9214d", - "resource": { - "resourceType": "Observation", - "id": "fabfb47f-c5e3-42ec-9c96-0e3760a9214d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 8.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49303ecf-8654-40c8-a42f-1920ec4a730f", - "resource": { - "resourceType": "Observation", - "id": "49303ecf-8654-40c8-a42f-1920ec4a730f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 140.78, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c923c15-2f88-4160-9ddc-12c29dfb9ddb", - "resource": { - "resourceType": "Observation", - "id": "0c923c15-2f88-4160-9ddc-12c29dfb9ddb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.79, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb3fff56-00ab-449b-8bc0-37bac09fcaf0", - "resource": { - "resourceType": "Observation", - "id": "cb3fff56-00ab-449b-8bc0-37bac09fcaf0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 102.71, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50a33072-716e-487c-9937-c680d25c5611", - "resource": { - "resourceType": "Observation", - "id": "50a33072-716e-487c-9937-c680d25c5611", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 23.04, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b371c06-ad8e-4fff-95a7-1eda63211530", - "resource": { - "resourceType": "Observation", - "id": "0b371c06-ad8e-4fff-95a7-1eda63211530", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 169.45, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:243016dc-80a3-4051-b09a-29c4bcb8bd00", - "resource": { - "resourceType": "Observation", - "id": "243016dc-80a3-4051-b09a-29c4bcb8bd00", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 143.48, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", - "resource": { - "resourceType": "Observation", - "id": "4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 64.11, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30610131-6e4c-45ff-8f9e-e146cea910e4", - "resource": { - "resourceType": "Observation", - "id": "30610131-6e4c-45ff-8f9e-e146cea910e4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 76.64, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:55998799-c7ce-4284-b9e0-ce9401537c9a", - "resource": { - "resourceType": "Observation", - "id": "55998799-c7ce-4284-b9e0-ce9401537c9a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5839414d-a5c6-42d1-a580-11ff69a5089f", - "resource": { - "resourceType": "Observation", - "id": "5839414d-a5c6-42d1-a580-11ff69a5089f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.9, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b18c6bef-9d11-452f-999c-326f2bb47fc9", - "resource": { - "resourceType": "MedicationRequest", - "id": "b18c6bef-9d11-452f-999c-326f2bb47fc9", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "authoredOn": "2010-10-18T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:4fd918ac-d1f7-4dec-9cb4-9ac3d9a6bb66", - "resource": { - "resourceType": "Claim", - "id": "4fd918ac-d1f7-4dec-9cb4-9ac3d9a6bb66", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "created": "2010-10-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:b18c6bef-9d11-452f-999c-326f2bb47fc9" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:143f7dc2-52c7-4d88-b91b-f9f0a2f25871", - "resource": { - "resourceType": "Immunization", - "id": "143f7dc2-52c7-4d88-b91b-f9f0a2f25871", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "occurrenceDateTime": "2010-10-18T05:15:51-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d83664cb-16d7-4b34-b292-df1701735e1b", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d83664cb-16d7-4b34-b292-df1701735e1b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:4e0ef354-df58-4853-b411-d4f5a8f89076", - "display": "Glucose" - }, - { - "reference": "urn:uuid:7866c71e-5535-4847-b40a-e612c3a7bb98", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:b6c369d1-c352-489f-8c55-8d5bcb4c59c7", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:fabfb47f-c5e3-42ec-9c96-0e3760a9214d", - "display": "Calcium" - }, - { - "reference": "urn:uuid:49303ecf-8654-40c8-a42f-1920ec4a730f", - "display": "Sodium" - }, - { - "reference": "urn:uuid:0c923c15-2f88-4160-9ddc-12c29dfb9ddb", - "display": "Potassium" - }, - { - "reference": "urn:uuid:cb3fff56-00ab-449b-8bc0-37bac09fcaf0", - "display": "Chloride" - }, - { - "reference": "urn:uuid:50a33072-716e-487c-9937-c680d25c5611", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:d402ed47-c431-42b6-bbb5-909171ab56d5", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d402ed47-c431-42b6-bbb5-909171ab56d5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - }, - "effectiveDateTime": "2010-10-18T05:15:51-07:00", - "issued": "2010-10-18T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:0b371c06-ad8e-4fff-95a7-1eda63211530", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:243016dc-80a3-4051-b09a-29c4bcb8bd00", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:4b35e110-5b54-4e0c-a891-b58bd0c7b2e5", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:30610131-6e4c-45ff-8f9e-e146cea910e4", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:ea6972ba-efb0-40a7-9e89-79bb7431510f", - "resource": { - "resourceType": "Claim", - "id": "ea6972ba-efb0-40a7-9e89-79bb7431510f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "created": "2010-10-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:143f7dc2-52c7-4d88-b91b-f9f0a2f25871" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:66468a95-8439-476d-ad23-f87d92f5a129", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "66468a95-8439-476d-ad23-f87d92f5a129", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ea6972ba-efb0-40a7-9e89-79bb7431510f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2010-10-18T05:30:51-07:00", - "end": "2011-10-18T05:30:51-07:00" - }, - "created": "2010-10-18T05:30:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ea6972ba-efb0-40a7-9e89-79bb7431510f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1222f480-de35-4581-8234-aac3e0aaebf6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2010-10-18T05:15:51-07:00", - "end": "2010-10-18T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560", - "resource": { - "resourceType": "Encounter", - "id": "e14a3dae-6bf7-4ecb-8c51-1ef2ff792560", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:da08dc9f-0ccd-4461-b501-3a0f5b0e3bce", - "resource": { - "resourceType": "Observation", - "id": "da08dc9f-0ccd-4461-b501-3a0f5b0e3bce", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff00b5aa-222d-4305-a633-13f637f87570", - "resource": { - "resourceType": "Observation", - "id": "ff00b5aa-222d-4305-a633-13f637f87570", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bcd47440-b08b-46fa-a255-d7890f350f94", - "resource": { - "resourceType": "Observation", - "id": "bcd47440-b08b-46fa-a255-d7890f350f94", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82475d55-eed8-4eca-ae6d-3da2a84b3a62", - "resource": { - "resourceType": "Observation", - "id": "82475d55-eed8-4eca-ae6d-3da2a84b3a62", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e561ae01-7419-4b58-b3a5-ea257b672791", - "resource": { - "resourceType": "Observation", - "id": "e561ae01-7419-4b58-b3a5-ea257b672791", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b92349e5-53ee-4b62-9f0d-a5096eee67b8", - "resource": { - "resourceType": "Observation", - "id": "b92349e5-53ee-4b62-9f0d-a5096eee67b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 84, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bace38ef-8a1d-4830-839c-70c64d7a6eaf", - "resource": { - "resourceType": "Observation", - "id": "bace38ef-8a1d-4830-839c-70c64d7a6eaf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", - "resource": { - "resourceType": "Observation", - "id": "c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 67.06, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7aab2a83-5c72-4bd5-bfae-e6cfff55725e", - "resource": { - "resourceType": "Observation", - "id": "7aab2a83-5c72-4bd5-bfae-e6cfff55725e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 8.68, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", - "resource": { - "resourceType": "Observation", - "id": "47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.75, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:62345264-3bbe-4cd9-8426-e66a2046ff45", - "resource": { - "resourceType": "Observation", - "id": "62345264-3bbe-4cd9-8426-e66a2046ff45", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.11, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", - "resource": { - "resourceType": "Observation", - "id": "ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 137.01, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9969b2b5-880a-4281-9f4f-1946b4028fa9", - "resource": { - "resourceType": "Observation", - "id": "9969b2b5-880a-4281-9f4f-1946b4028fa9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.09, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47f2462f-7edc-43c7-af3d-2664d3ccfc32", - "resource": { - "resourceType": "Observation", - "id": "47f2462f-7edc-43c7-af3d-2664d3ccfc32", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 106.45, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f73039a-d34c-4368-b9b6-a4e51fb69b97", - "resource": { - "resourceType": "Observation", - "id": "9f73039a-d34c-4368-b9b6-a4e51fb69b97", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 22.78, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1780db0a-b024-49b7-8f1d-4909f8a90f21", - "resource": { - "resourceType": "Observation", - "id": "1780db0a-b024-49b7-8f1d-4909f8a90f21", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:75b75ae3-8ac2-4f47-9d62-26b05f9f1600", - "resource": { - "resourceType": "Observation", - "id": "75b75ae3-8ac2-4f47-9d62-26b05f9f1600", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "valueQuantity": { - "value": 6.26, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:46bbd862-0fd4-4f10-abd5-16695c303df3", - "resource": { - "resourceType": "Procedure", - "id": "46bbd862-0fd4-4f10-abd5-16695c303df3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "performedPeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:30:51-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:dda7c2c4-a680-428b-bf27-4767b7b25ce0", - "resource": { - "resourceType": "MedicationRequest", - "id": "dda7c2c4-a680-428b-bf27-4767b7b25ce0", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "authoredOn": "2011-10-24T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:228d1384-5329-4bfd-87d3-5c1844b42517", - "resource": { - "resourceType": "Claim", - "id": "228d1384-5329-4bfd-87d3-5c1844b42517", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "created": "2011-10-24T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:dda7c2c4-a680-428b-bf27-4767b7b25ce0" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:191dbed8-5356-499f-8bcb-1f565f14b47d", - "resource": { - "resourceType": "Immunization", - "id": "191dbed8-5356-499f-8bcb-1f565f14b47d", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "occurrenceDateTime": "2011-10-24T05:15:51-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0f6c87ca-1a69-4ec2-b698-c37f5a4b0a46", - "resource": { - "resourceType": "DiagnosticReport", - "id": "0f6c87ca-1a69-4ec2-b698-c37f5a4b0a46", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - }, - "effectiveDateTime": "2011-10-24T05:15:51-07:00", - "issued": "2011-10-24T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:c0e6cb4d-a9d1-413b-af3d-b5db341c1cf5", - "display": "Glucose" - }, - { - "reference": "urn:uuid:7aab2a83-5c72-4bd5-bfae-e6cfff55725e", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:47a7aa3a-f5e5-42a0-b3a9-4adeabcfd1f3", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:62345264-3bbe-4cd9-8426-e66a2046ff45", - "display": "Calcium" - }, - { - "reference": "urn:uuid:ca6f0aa1-54d0-4bee-b073-3a12dc62f2d8", - "display": "Sodium" - }, - { - "reference": "urn:uuid:9969b2b5-880a-4281-9f4f-1946b4028fa9", - "display": "Potassium" - }, - { - "reference": "urn:uuid:47f2462f-7edc-43c7-af3d-2664d3ccfc32", - "display": "Chloride" - }, - { - "reference": "urn:uuid:9f73039a-d34c-4368-b9b6-a4e51fb69b97", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:5410ce3a-7f8f-40da-bc01-e4d5432967e2", - "resource": { - "resourceType": "Claim", - "id": "5410ce3a-7f8f-40da-bc01-e4d5432967e2", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "created": "2011-10-24T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:191dbed8-5356-499f-8bcb-1f565f14b47d" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:46bbd862-0fd4-4f10-abd5-16695c303df3" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 455.84, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:73f5da41-ea78-4cf7-9d9e-63187ddd30bb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "73f5da41-ea78-4cf7-9d9e-63187ddd30bb", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5410ce3a-7f8f-40da-bc01-e4d5432967e2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2011-10-24T05:45:51-07:00", - "end": "2012-10-24T05:45:51-07:00" - }, - "created": "2011-10-24T05:45:51-07:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5410ce3a-7f8f-40da-bc01-e4d5432967e2" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e14a3dae-6bf7-4ecb-8c51-1ef2ff792560" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2011-10-24T05:15:51-07:00", - "end": "2011-10-24T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 455.84, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 91.168, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 364.672, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 455.84, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 455.84, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 477.088, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244", - "resource": { - "resourceType": "Encounter", - "id": "ccd45c67-85e0-48af-be78-95d9547f9244", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", - "resource": { - "resourceType": "Condition", - "id": "b6596ab7-d051-4237-a4d0-1d86c89e92b8", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68496003", - "display": "Polyp of colon" - } - ], - "text": "Polyp of colon" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "onsetDateTime": "2011-12-06T04:15:51-08:00", - "recordedDate": "2011-12-06T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b24b6a98-1bb8-4da9-87f4-9b099c008204", - "resource": { - "resourceType": "Observation", - "id": "b24b6a98-1bb8-4da9-87f4-9b099c008204", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33756-8", - "display": "Polyp size greatest dimension by CAP cancer protocols" - } - ], - "text": "Polyp size greatest dimension by CAP cancer protocols" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "effectiveDateTime": "2011-12-06T04:15:51-08:00", - "issued": "2011-12-06T04:15:51.927-08:00", - "valueQuantity": { - "value": 5.618, - "unit": "mm", - "system": "http://unitsofmeasure.org", - "code": "mm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6fa5382a-c18f-4e03-a616-3caa504a88ba", - "resource": { - "resourceType": "Observation", - "id": "6fa5382a-c18f-4e03-a616-3caa504a88ba", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57905-2", - "display": "Hemoglobin.gastrointestinal [Presence] in Stool by Immunologic method" - } - ], - "text": "Hemoglobin.gastrointestinal [Presence] in Stool by Immunologic method" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "effectiveDateTime": "2011-12-06T04:15:51-08:00", - "issued": "2011-12-06T04:15:51.927-08:00", - "valueQuantity": { - "value": 13.654, - "unit": "ng/mL", - "system": "http://unitsofmeasure.org", - "code": "ng/mL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ead9552-589f-41e0-a094-c344a46b355c", - "resource": { - "resourceType": "Procedure", - "id": "5ead9552-589f-41e0-a094-c344a46b355c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "performedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T04:42:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:61ad0c64-16f0-4b26-8011-97dc3aa65eb8", - "resource": { - "resourceType": "Procedure", - "id": "61ad0c64-16f0-4b26-8011-97dc3aa65eb8", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104435004", - "display": "Screening for occult blood in feces (procedure)" - } - ], - "text": "Screening for occult blood in feces (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "performedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T04:30:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", - "display": "Polyp of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:fef09971-e75e-40b6-a6db-5d722d89ff3d", - "resource": { - "resourceType": "Procedure", - "id": "fef09971-e75e-40b6-a6db-5d722d89ff3d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - }, - "performedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T04:30:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8", - "display": "Polyp of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a6149938-4660-49a1-af81-ed03a6f62c3b", - "resource": { - "resourceType": "Claim", - "id": "a6149938-4660-49a1-af81-ed03a6f62c3b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "created": "2011-12-06T05:27:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:5ead9552-589f-41e0-a094-c344a46b355c" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:61ad0c64-16f0-4b26-8011-97dc3aa65eb8" - } - }, - { - "sequence": 3, - "procedureReference": { - "reference": "urn:uuid:fef09971-e75e-40b6-a6db-5d722d89ff3d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 8196.11, - "currency": "USD" - } - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68496003", - "display": "Polyp of colon" - } - ], - "text": "Polyp of colon" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104435004", - "display": "Screening for occult blood in feces (procedure)" - } - ], - "text": "Screening for occult blood in feces (procedure)" - }, - "net": { - "value": 8462.10, - "currency": "USD" - } - }, - { - "sequence": 5, - "procedureSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "net": { - "value": 11956.73, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0768d249-9d7f-4ab9-86aa-59401ad5f4a1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0768d249-9d7f-4ab9-86aa-59401ad5f4a1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "UnitedHealthcare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "UnitedHealthcare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a6149938-4660-49a1-af81-ed03a6f62c3b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2011-12-06T05:27:51-08:00", - "end": "2012-12-06T05:27:51-08:00" - }, - "created": "2011-12-06T05:27:51-08:00", - "insurer": { - "display": "UnitedHealthcare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a6149938-4660-49a1-af81-ed03a6f62c3b" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b6596ab7-d051-4237-a4d0-1d86c89e92b8" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "UnitedHealthcare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ccd45c67-85e0-48af-be78-95d9547f9244" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 8196.11, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1639.2220000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 6556.888000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 8196.11, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 8196.11, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "68496003", - "display": "Polyp of colon" - } - ], - "text": "Polyp of colon" - }, - "servicedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "104435004", - "display": "Screening for occult blood in feces (procedure)" - } - ], - "text": "Screening for occult blood in feces (procedure)" - }, - "servicedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 8462.10, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1692.42, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 6769.68, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 8462.10, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 8462.10, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "servicedPeriod": { - "start": "2011-12-06T04:15:51-08:00", - "end": "2011-12-06T05:27:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 11956.73, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2391.346, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9565.384, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11956.73, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11956.73, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 22891.952, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f", - "resource": { - "resourceType": "Encounter", - "id": "3f5eebb6-0f33-4823-9858-0ab6d00bd21f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d4e39b43-4b12-4349-8094-bdd382316a5b", - "resource": { - "resourceType": "Observation", - "id": "d4e39b43-4b12-4349-8094-bdd382316a5b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d700845-89e0-4185-998b-fb008c222877", - "resource": { - "resourceType": "Observation", - "id": "6d700845-89e0-4185-998b-fb008c222877", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:257843c1-2d0e-44c2-873b-1e8d1344719a", - "resource": { - "resourceType": "Observation", - "id": "257843c1-2d0e-44c2-873b-1e8d1344719a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6890854a-49b2-4ba5-aa99-864b24664d0d", - "resource": { - "resourceType": "Observation", - "id": "6890854a-49b2-4ba5-aa99-864b24664d0d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:74976a7f-104a-472f-87be-03bbc3b93bd4", - "resource": { - "resourceType": "Observation", - "id": "74976a7f-104a-472f-87be-03bbc3b93bd4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e195dd89-9de4-4908-a001-866836151491", - "resource": { - "resourceType": "Observation", - "id": "e195dd89-9de4-4908-a001-866836151491", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 69, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ce435a3e-fa1d-47d5-9665-240da052d3ef", - "resource": { - "resourceType": "Observation", - "id": "ce435a3e-fa1d-47d5-9665-240da052d3ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6a40303d-051f-4fed-9607-d5ae00c08a4c", - "resource": { - "resourceType": "Observation", - "id": "6a40303d-051f-4fed-9607-d5ae00c08a4c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 83.7, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b27f29c-7faf-4197-a2ad-5350aaece63e", - "resource": { - "resourceType": "Observation", - "id": "6b27f29c-7faf-4197-a2ad-5350aaece63e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.98, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:44167af7-c65a-468c-8955-09709335d754", - "resource": { - "resourceType": "Observation", - "id": "44167af7-c65a-468c-8955-09709335d754", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.74, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", - "resource": { - "resourceType": "Observation", - "id": "6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 8.88, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:06c5782f-7435-4cf1-bb98-b0098630e4ea", - "resource": { - "resourceType": "Observation", - "id": "06c5782f-7435-4cf1-bb98-b0098630e4ea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 141.29, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:638739d4-0337-4127-af89-0ece8620a94f", - "resource": { - "resourceType": "Observation", - "id": "638739d4-0337-4127-af89-0ece8620a94f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.9, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", - "resource": { - "resourceType": "Observation", - "id": "c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 110.69, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a3920911-4744-44b8-973f-295ece827da7", - "resource": { - "resourceType": "Observation", - "id": "a3920911-4744-44b8-973f-295ece827da7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 20.17, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6fdd33ad-3a56-4711-bf57-801ea954c9a1", - "resource": { - "resourceType": "Observation", - "id": "6fdd33ad-3a56-4711-bf57-801ea954c9a1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65411e70-da5e-4137-866c-36291ce21747", - "resource": { - "resourceType": "Observation", - "id": "65411e70-da5e-4137-866c-36291ce21747", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.9, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed7806ea-1108-4cc4-8fb8-3ec4ffb30257", - "resource": { - "resourceType": "Procedure", - "id": "ed7806ea-1108-4cc4-8fb8-3ec4ffb30257", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "performedPeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:30:51-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:fdd51ea4-9921-4dc1-9acb-529172952993", - "resource": { - "resourceType": "MedicationRequest", - "id": "fdd51ea4-9921-4dc1-9acb-529172952993", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "authoredOn": "2012-10-29T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:b3a13b4b-06ed-4322-aa7d-7f261e764707", - "resource": { - "resourceType": "Claim", - "id": "b3a13b4b-06ed-4322-aa7d-7f261e764707", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "created": "2012-10-29T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:fdd51ea4-9921-4dc1-9acb-529172952993" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:370c4480-eebf-43df-9a05-fcba4c17d12b", - "resource": { - "resourceType": "Immunization", - "id": "370c4480-eebf-43df-9a05-fcba4c17d12b", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "occurrenceDateTime": "2012-10-29T05:15:51-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:61ce6171-9054-4aa8-ae88-572486a0fe48", - "resource": { - "resourceType": "Immunization", - "id": "61ce6171-9054-4aa8-ae88-572486a0fe48", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "occurrenceDateTime": "2012-10-29T05:15:51-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:cff7f7ea-206c-44d9-8e14-a79145327cb8", - "resource": { - "resourceType": "DiagnosticReport", - "id": "cff7f7ea-206c-44d9-8e14-a79145327cb8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - }, - "effectiveDateTime": "2012-10-29T05:15:51-07:00", - "issued": "2012-10-29T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:6a40303d-051f-4fed-9607-d5ae00c08a4c", - "display": "Glucose" - }, - { - "reference": "urn:uuid:6b27f29c-7faf-4197-a2ad-5350aaece63e", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:44167af7-c65a-468c-8955-09709335d754", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:6f7e8c87-b8ee-4e62-8a72-a5e8e0ef19b3", - "display": "Calcium" - }, - { - "reference": "urn:uuid:06c5782f-7435-4cf1-bb98-b0098630e4ea", - "display": "Sodium" - }, - { - "reference": "urn:uuid:638739d4-0337-4127-af89-0ece8620a94f", - "display": "Potassium" - }, - { - "reference": "urn:uuid:c0ca6a9a-e3c6-4f10-9361-4ee71b00fb87", - "display": "Chloride" - }, - { - "reference": "urn:uuid:a3920911-4744-44b8-973f-295ece827da7", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4", - "resource": { - "resourceType": "Claim", - "id": "718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "created": "2012-10-29T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:370c4480-eebf-43df-9a05-fcba4c17d12b" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:61ce6171-9054-4aa8-ae88-572486a0fe48" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ed7806ea-1108-4cc4-8fb8-3ec4ffb30257" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 938.72, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5b5e55bf-ecb1-45cf-8836-f572c7d1d443", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5b5e55bf-ecb1-45cf-8836-f572c7d1d443", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2012-10-29T05:45:51-07:00", - "end": "2013-10-29T05:45:51-07:00" - }, - "created": "2012-10-29T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:718c3dd4-bd6a-4c9a-9dbb-f55380a36eb4" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3f5eebb6-0f33-4823-9858-0ab6d00bd21f" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "servicedPeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2012-10-29T05:15:51-07:00", - "end": "2012-10-29T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 938.72, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 187.74400000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 750.9760000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 938.72, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 938.72, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 975.8080000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935", - "resource": { - "resourceType": "Encounter", - "id": "2ce6f12f-85b2-4092-8b24-d91d66313935", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758", - "resource": { - "resourceType": "Condition", - "id": "9e17df71-4a5a-4ce5-ae43-9ca30c031758", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "713197008", - "display": "Recurrent rectal polyp" - } - ], - "text": "Recurrent rectal polyp" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - }, - "onsetDateTime": "2013-01-29T04:15:51-08:00", - "recordedDate": "2013-01-29T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b02a3e53-1eda-4624-8607-f780cd226880", - "resource": { - "resourceType": "Observation", - "id": "b02a3e53-1eda-4624-8607-f780cd226880", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - }, - "effectiveDateTime": "2013-01-29T04:15:51-08:00", - "issued": "2013-01-29T04:15:51.927-08:00", - "valueQuantity": { - "value": 6, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ea277578-dab8-479d-8e25-bcf8eb7dc7a3", - "resource": { - "resourceType": "Procedure", - "id": "ea277578-dab8-479d-8e25-bcf8eb7dc7a3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - }, - "performedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T04:52:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e0a9b65b-019b-47fe-9bba-9b6cd3a14857", - "resource": { - "resourceType": "Procedure", - "id": "e0a9b65b-019b-47fe-9bba-9b6cd3a14857", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - }, - "performedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T04:30:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758", - "display": "Recurrent rectal polyp" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d7e5b0cd-099e-472d-bb18-722f3f77b6a4", - "resource": { - "resourceType": "Claim", - "id": "d7e5b0cd-099e-472d-bb18-722f3f77b6a4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "created": "2013-01-29T05:22:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ea277578-dab8-479d-8e25-bcf8eb7dc7a3" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:e0a9b65b-019b-47fe-9bba-9b6cd3a14857" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 23036.39, - "currency": "USD" - } - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "713197008", - "display": "Recurrent rectal polyp" - } - ], - "text": "Recurrent rectal polyp" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "net": { - "value": 12474.97, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6c835e0f-b9f1-4253-8aad-b01473276b07", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6c835e0f-b9f1-4253-8aad-b01473276b07", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d7e5b0cd-099e-472d-bb18-722f3f77b6a4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-01-29T05:22:51-08:00", - "end": "2014-01-29T05:22:51-08:00" - }, - "created": "2013-01-29T05:22:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:d7e5b0cd-099e-472d-bb18-722f3f77b6a4" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e17df71-4a5a-4ce5-ae43-9ca30c031758" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2ce6f12f-85b2-4092-8b24-d91d66313935" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 23036.39, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 4607.278, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 18429.112, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 23036.39, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 23036.39, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "713197008", - "display": "Recurrent rectal polyp" - } - ], - "text": "Recurrent rectal polyp" - }, - "servicedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "274031008", - "display": "Rectal polypectomy" - } - ], - "text": "Rectal polypectomy" - }, - "servicedPeriod": { - "start": "2013-01-29T04:15:51-08:00", - "end": "2013-01-29T05:22:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 12474.97, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2494.994, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9979.976, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 12474.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 12474.97, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 28409.088000000003, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b", - "resource": { - "resourceType": "Encounter", - "id": "2fd76fad-d839-43b0-9bbf-2adc6a15983b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2013-02-01T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2013-02-01T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "resource": { - "resourceType": "Condition", - "id": "a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ], - "text": "Overlapping malignant neoplasm of colon" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" - }, - "onsetDateTime": "2013-02-01T04:15:51-08:00", - "abatementDateTime": "2015-07-23T05:15:51-07:00", - "recordedDate": "2013-02-01T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:2dd59121-7bfa-41f2-83e6-8c9ebabc74d9", - "resource": { - "resourceType": "CareTeam", - "id": "2dd59121-7bfa-41f2-83e6-8c9ebabc74d9", - "status": "inactive", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" - }, - "period": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2015-07-23T05:15:51-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ], - "text": "Overlapping malignant neoplasm of colon" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:49485686-7da2-4c6e-b039-14fd7c78f02c", - "resource": { - "resourceType": "CarePlan", - "id": "49485686-7da2-4c6e-b039-14fd7c78f02c", - "text": { - "status": "generated", - "div": "
    Care Plan for Cancer care plan.
    Activities:
    • Cancer care plan
    • Cancer care plan
    • Cancer care plan

    Care plan is meant to treat Overlapping malignant neoplasm of colon.
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "395082007", - "display": "Cancer care plan" - } - ], - "text": "Cancer care plan" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" - }, - "period": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2015-07-23T05:15:51-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:2dd59121-7bfa-41f2-83e6-8c9ebabc74d9" - } - ], - "addresses": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "226234005", - "display": "Healthy diet" - } - ], - "text": "Healthy diet" - }, - "status": "completed", - "location": { - "display": "CAPE COD HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703993001", - "display": "Colonoscopy planned" - } - ], - "text": "Colonoscopy planned" - }, - "status": "completed", - "location": { - "display": "CAPE COD HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "243072006", - "display": "Cancer education" - } - ], - "text": "Cancer education" - }, - "status": "completed", - "location": { - "display": "CAPE COD HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:792060bc-036f-4266-a268-cf45426684cb", - "resource": { - "resourceType": "Claim", - "id": "792060bc-036f-4266-a268-cf45426684cb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2013-02-01T04:30:51-08:00" - }, - "created": "2013-02-01T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ], - "text": "Overlapping malignant neoplasm of colon" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:11508325-4f1d-4d58-b692-579080484764", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "11508325-4f1d-4d58-b692-579080484764", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "792060bc-036f-4266-a268-cf45426684cb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-02-01T04:30:51-08:00", - "end": "2014-02-01T04:30:51-08:00" - }, - "created": "2013-02-01T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:792060bc-036f-4266-a268-cf45426684cb" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2013-02-01T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2fd76fad-d839-43b0-9bbf-2adc6a15983b" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ], - "text": "Overlapping malignant neoplasm of colon" - }, - "servicedPeriod": { - "start": "2013-02-01T04:15:51-08:00", - "end": "2013-02-01T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403", - "resource": { - "resourceType": "Encounter", - "id": "4809ff01-1e8c-42d2-a7f1-32f786aaa403", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-09T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-09T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cc9e49a5-5011-4785-9674-e75cd8f6834c", - "resource": { - "resourceType": "Observation", - "id": "cc9e49a5-5011-4785-9674-e75cd8f6834c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - }, - "effectiveDateTime": "2013-02-08T04:15:51-08:00", - "issued": "2013-02-08T04:15:51.927-08:00", - "valueQuantity": { - "value": 6, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:313e470a-a6a1-4e8c-9feb-0ff493f359bb", - "resource": { - "resourceType": "Procedure", - "id": "313e470a-a6a1-4e8c-9feb-0ff493f359bb", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43075005", - "display": "Partial resection of colon" - } - ], - "text": "Partial resection of colon" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - }, - "performedPeriod": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-08T04:30:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a4792bae-b76e-48b9-9f34-4de61a60bdbb", - "resource": { - "resourceType": "CareTeam", - "id": "a4792bae-b76e-48b9-9f34-4de61a60bdbb", - "status": "inactive", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - }, - "period": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-19T04:15:51-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ], - "text": "Overlapping malignant neoplasm of colon" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:a9820028-405f-459c-b510-fae895c22f47", - "resource": { - "resourceType": "CarePlan", - "id": "a9820028-405f-459c-b510-fae895c22f47", - "text": { - "status": "generated", - "div": "
    Care Plan for Major surgery care management.
    Activities:
    • Major surgery care management
    • Major surgery care management

    Care plan is meant to treat Overlapping malignant neoplasm of colon.
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "781831000000109", - "display": "Major surgery care management" - } - ], - "text": "Major surgery care management" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - }, - "period": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-19T04:15:51-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:a4792bae-b76e-48b9-9f34-4de61a60bdbb" - } - ], - "addresses": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10888001", - "display": "Liquid diet" - } - ], - "text": "Liquid diet" - }, - "status": "completed", - "location": { - "display": "CAPE COD HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "103744005", - "display": "Administration of intravenous fluids" - } - ], - "text": "Administration of intravenous fluids" - }, - "status": "completed", - "location": { - "display": "CAPE COD HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:010e4ade-c95c-4b50-a84b-cf66f3893ddf", - "resource": { - "resourceType": "Claim", - "id": "010e4ade-c95c-4b50-a84b-cf66f3893ddf", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-09T04:30:51-08:00" - }, - "created": "2013-02-09T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:313e470a-a6a1-4e8c-9feb-0ff493f359bb" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43075005", - "display": "Partial resection of colon" - } - ], - "text": "Partial resection of colon" - }, - "net": { - "value": 38424.92, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:db5b770b-726c-40fd-bac2-795664a67023", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "db5b770b-726c-40fd-bac2-795664a67023", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "010e4ade-c95c-4b50-a84b-cf66f3893ddf" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-02-09T04:30:51-08:00", - "end": "2014-02-09T04:30:51-08:00" - }, - "created": "2013-02-09T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:010e4ade-c95c-4b50-a84b-cf66f3893ddf" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-09T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4809ff01-1e8c-42d2-a7f1-32f786aaa403" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43075005", - "display": "Partial resection of colon" - } - ], - "text": "Partial resection of colon" - }, - "servicedPeriod": { - "start": "2013-02-08T04:15:51-08:00", - "end": "2013-02-09T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 38424.92, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 7684.984, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 30739.936, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 38424.92, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 38424.92, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 30739.936, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476", - "resource": { - "resourceType": "Encounter", - "id": "2d926120-545e-45bd-bb18-1bd83b3ec476", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a5f4315a-3bb6-416e-b44f-c7a6f938ac6c", - "resource": { - "resourceType": "Observation", - "id": "a5f4315a-3bb6-416e-b44f-c7a6f938ac6c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 6, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6d01daa-1018-4dbc-9b2c-8a0a24029d25", - "resource": { - "resourceType": "Observation", - "id": "b6d01daa-1018-4dbc-9b2c-8a0a24029d25", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 89.55, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:55124c0f-5b70-45ea-b4a9-30cc4eb883ba", - "resource": { - "resourceType": "Observation", - "id": "55124c0f-5b70-45ea-b4a9-30cc4eb883ba", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 10.77, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:02a11d04-ad54-4bfa-9bd6-f57760863136", - "resource": { - "resourceType": "Observation", - "id": "02a11d04-ad54-4bfa-9bd6-f57760863136", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 2.5003, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebd2b5d8-5411-43de-bf84-d065d60e6986", - "resource": { - "resourceType": "Observation", - "id": "ebd2b5d8-5411-43de-bf84-d065d60e6986", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.5, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53a0e1bc-650e-4dfb-9ba4-74c211160b03", - "resource": { - "resourceType": "Observation", - "id": "53a0e1bc-650e-4dfb-9ba4-74c211160b03", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 138.42, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36c36666-4a04-415c-9951-dbbcd5e3c631", - "resource": { - "resourceType": "Observation", - "id": "36c36666-4a04-415c-9951-dbbcd5e3c631", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.52, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:38de03a9-5ee7-41fb-86ca-1a95b507bbd7", - "resource": { - "resourceType": "Observation", - "id": "38de03a9-5ee7-41fb-86ca-1a95b507bbd7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 102.93, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", - "resource": { - "resourceType": "Observation", - "id": "4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 23.88, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77456a94-0af1-4cdd-86e7-e3748082cecc", - "resource": { - "resourceType": "Observation", - "id": "77456a94-0af1-4cdd-86e7-e3748082cecc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 17.566, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db36842c-234e-4f37-bba9-9e60b5ed6c7d", - "resource": { - "resourceType": "Observation", - "id": "db36842c-234e-4f37-bba9-9e60b5ed6c7d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 64.571, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d57e0fc-8f13-49b0-a9e8-76de6c37850c", - "resource": { - "resourceType": "Observation", - "id": "2d57e0fc-8f13-49b0-a9e8-76de6c37850c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 5.2204, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4ad163fb-192a-4af3-b4bf-94f439510b9b", - "resource": { - "resourceType": "Observation", - "id": "4ad163fb-192a-4af3-b4bf-94f439510b9b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 3.1771, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2e07e21f-5869-4904-ac49-6de060667641", - "resource": { - "resourceType": "Observation", - "id": "2e07e21f-5869-4904-ac49-6de060667641", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.18474, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1aa9cbcb-216c-4e4e-ac58-95432082003f", - "resource": { - "resourceType": "Observation", - "id": "1aa9cbcb-216c-4e4e-ac58-95432082003f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 133.07, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e3bcd75-77c9-4302-b1c6-556595550cd1", - "resource": { - "resourceType": "Observation", - "id": "3e3bcd75-77c9-4302-b1c6-556595550cd1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 51.686, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", - "resource": { - "resourceType": "Observation", - "id": "bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 12.513, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8c6a2800-a7d8-4471-906b-22a3fb51b313", - "resource": { - "resourceType": "Observation", - "id": "8c6a2800-a7d8-4471-906b-22a3fb51b313", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 2.6884, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ded636c1-e6a1-4f0f-8fe2-85a95748236a", - "resource": { - "resourceType": "Observation", - "id": "ded636c1-e6a1-4f0f-8fe2-85a95748236a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.6605, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4f8dee98-f03f-4fc5-b7dd-d9841c921164", - "resource": { - "resourceType": "Observation", - "id": "4f8dee98-f03f-4fc5-b7dd-d9841c921164", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 12.236, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d06a97e8-696f-4a05-a5e1-b0a962296fab", - "resource": { - "resourceType": "Observation", - "id": "d06a97e8-696f-4a05-a5e1-b0a962296fab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.069, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f535920-cd6d-4842-a1eb-2dbd73214b17", - "resource": { - "resourceType": "Observation", - "id": "9f535920-cd6d-4842-a1eb-2dbd73214b17", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 90.317, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c6a078c-2a08-47b2-93f3-689ff7e2987f", - "resource": { - "resourceType": "Observation", - "id": "9c6a078c-2a08-47b2-93f3-689ff7e2987f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 32.23, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", - "resource": { - "resourceType": "Observation", - "id": "f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 35.221, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d3e1e40b-5743-4ed5-b047-3f548131ded9", - "resource": { - "resourceType": "Observation", - "id": "d3e1e40b-5743-4ed5-b047-3f548131ded9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 41.158, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", - "resource": { - "resourceType": "Observation", - "id": "5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 438.01, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:08e428c1-6d81-4ea1-bd75-d33265bbd37d", - "resource": { - "resourceType": "Observation", - "id": "08e428c1-6d81-4ea1-bd75-d33265bbd37d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 273.43, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecf51aa3-5928-418d-9411-72618c5298fb", - "resource": { - "resourceType": "Observation", - "id": "ecf51aa3-5928-418d-9411-72618c5298fb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "valueQuantity": { - "value": 10.09, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25f6ccb3-6e71-41a1-a3ff-97ad95940745", - "resource": { - "resourceType": "Procedure", - "id": "25f6ccb3-6e71-41a1-a3ff-97ad95940745", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "performedPeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:30:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a3d098ca-f27c-497d-b484-e78f34e7652b", - "resource": { - "resourceType": "MedicationRequest", - "id": "a3d098ca-f27c-497d-b484-e78f34e7652b", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1803932", - "display": "Leucovorin 100 MG Injection" - } - ], - "text": "Leucovorin 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "authoredOn": "2013-02-26T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5a30cba6-ec71-4a88-a6c5-6a2c4429218e", - "resource": { - "resourceType": "Claim", - "id": "5a30cba6-ec71-4a88-a6c5-6a2c4429218e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "created": "2013-02-26T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:a3d098ca-f27c-497d-b484-e78f34e7652b" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - } - ] - } - ], - "total": { - "value": 63.93, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:22394b83-909f-4c4c-8e72-195faf344c64", - "resource": { - "resourceType": "MedicationRequest", - "id": "22394b83-909f-4c4c-8e72-195faf344c64", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1736776", - "display": "10 ML oxaliplatin 5 MG/ML Injection" - } - ], - "text": "10 ML oxaliplatin 5 MG/ML Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "authoredOn": "2013-02-26T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:26da09d2-af4b-4ee4-945a-a3dda5fc9243", - "resource": { - "resourceType": "Claim", - "id": "26da09d2-af4b-4ee4-945a-a3dda5fc9243", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "created": "2013-02-26T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:22394b83-909f-4c4c-8e72-195faf344c64" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - } - ] - } - ], - "total": { - "value": 2507.24, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b7662618-21d8-4e39-b571-3a853e1b0d86", - "resource": { - "resourceType": "DiagnosticReport", - "id": "b7662618-21d8-4e39-b571-3a853e1b0d86", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:b6d01daa-1018-4dbc-9b2c-8a0a24029d25", - "display": "Glucose" - }, - { - "reference": "urn:uuid:55124c0f-5b70-45ea-b4a9-30cc4eb883ba", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:02a11d04-ad54-4bfa-9bd6-f57760863136", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:ebd2b5d8-5411-43de-bf84-d065d60e6986", - "display": "Calcium" - }, - { - "reference": "urn:uuid:53a0e1bc-650e-4dfb-9ba4-74c211160b03", - "display": "Sodium" - }, - { - "reference": "urn:uuid:36c36666-4a04-415c-9951-dbbcd5e3c631", - "display": "Potassium" - }, - { - "reference": "urn:uuid:38de03a9-5ee7-41fb-86ca-1a95b507bbd7", - "display": "Chloride" - }, - { - "reference": "urn:uuid:4d6b8fee-51c0-4a21-bf5c-c716e7ce096b", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:77456a94-0af1-4cdd-86e7-e3748082cecc", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:db36842c-234e-4f37-bba9-9e60b5ed6c7d", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:2d57e0fc-8f13-49b0-a9e8-76de6c37850c", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4ad163fb-192a-4af3-b4bf-94f439510b9b", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:2e07e21f-5869-4904-ac49-6de060667641", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:1aa9cbcb-216c-4e4e-ac58-95432082003f", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:3e3bcd75-77c9-4302-b1c6-556595550cd1", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:bdb3a6c5-2f9f-4122-8162-c6ca9573ecbb", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7dacebf4-2456-454a-b1ec-a804c4e69da2", - "resource": { - "resourceType": "DiagnosticReport", - "id": "7dacebf4-2456-454a-b1ec-a804c4e69da2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - }, - "effectiveDateTime": "2013-02-26T04:15:51-08:00", - "issued": "2013-02-26T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:8c6a2800-a7d8-4471-906b-22a3fb51b313", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:ded636c1-e6a1-4f0f-8fe2-85a95748236a", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:4f8dee98-f03f-4fc5-b7dd-d9841c921164", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:d06a97e8-696f-4a05-a5e1-b0a962296fab", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:9f535920-cd6d-4842-a1eb-2dbd73214b17", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:9c6a078c-2a08-47b2-93f3-689ff7e2987f", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:f3cc23f8-ab0f-4d47-9d47-2ccbe079cf57", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:d3e1e40b-5743-4ed5-b047-3f548131ded9", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:5c1ac4e2-8a69-45f8-8c76-e50d6b90f863", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:08e428c1-6d81-4ea1-bd75-d33265bbd37d", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:ecf51aa3-5928-418d-9411-72618c5298fb", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0752ea3e-d40e-40d1-b5c7-fc33171125ca", - "resource": { - "resourceType": "Claim", - "id": "0752ea3e-d40e-40d1-b5c7-fc33171125ca", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "created": "2013-02-26T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:25f6ccb3-6e71-41a1-a3ff-97ad95940745" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 7161.00, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8beec78a-24b2-4b47-abda-b837cfbb259f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8beec78a-24b2-4b47-abda-b837cfbb259f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0752ea3e-d40e-40d1-b5c7-fc33171125ca" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-02-26T04:45:51-08:00", - "end": "2014-02-26T04:45:51-08:00" - }, - "created": "2013-02-26T04:45:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0752ea3e-d40e-40d1-b5c7-fc33171125ca" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2d926120-545e-45bd-bb18-1bd83b3ec476" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-02-26T04:15:51-08:00", - "end": "2013-02-26T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 7161.00, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1432.2, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 5728.8, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 7161.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 7161.00, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 5728.8, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75", - "resource": { - "resourceType": "Encounter", - "id": "767da137-22c0-4811-8d09-af08dfda2f75", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:45:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:acaeefb1-9a5b-41b6-8f3a-5e5d3448fd5f", - "resource": { - "resourceType": "Observation", - "id": "acaeefb1-9a5b-41b6-8f3a-5e5d3448fd5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d37dc6f-e655-4d03-a64a-bfb7b32c4947", - "resource": { - "resourceType": "Observation", - "id": "7d37dc6f-e655-4d03-a64a-bfb7b32c4947", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 96.75, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2372beda-927e-460f-9de0-561b20d4c7a9", - "resource": { - "resourceType": "Observation", - "id": "2372beda-927e-460f-9de0-561b20d4c7a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 16.01, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c5cf511-b90d-4375-a10d-4b2c38c5e448", - "resource": { - "resourceType": "Observation", - "id": "1c5cf511-b90d-4375-a10d-4b2c38c5e448", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.8776, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8476f78-77fc-41dc-a1e1-482c0eb5a825", - "resource": { - "resourceType": "Observation", - "id": "b8476f78-77fc-41dc-a1e1-482c0eb5a825", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.48, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:75d3c18e-f2c0-4f63-b7d5-0a1000833542", - "resource": { - "resourceType": "Observation", - "id": "75d3c18e-f2c0-4f63-b7d5-0a1000833542", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 139.08, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71f65388-c251-40b6-bed7-72ce23d07ab5", - "resource": { - "resourceType": "Observation", - "id": "71f65388-c251-40b6-bed7-72ce23d07ab5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.8, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f9f36cee-0e35-4c5d-b8e1-f54c66269135", - "resource": { - "resourceType": "Observation", - "id": "f9f36cee-0e35-4c5d-b8e1-f54c66269135", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 106.95, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0161429-03f9-44c1-9271-2b5f55c7b4bb", - "resource": { - "resourceType": "Observation", - "id": "c0161429-03f9-44c1-9271-2b5f55c7b4bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 22.13, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25fd4b9c-4884-4704-b4c9-f7451c8e0869", - "resource": { - "resourceType": "Observation", - "id": "25fd4b9c-4884-4704-b4c9-f7451c8e0869", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 6.4972, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d981902d-784f-4484-8076-1321f1f5728e", - "resource": { - "resourceType": "Observation", - "id": "d981902d-784f-4484-8076-1321f1f5728e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 76.426, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4cf77f06-37f8-4a18-8424-08ff79bc6295", - "resource": { - "resourceType": "Observation", - "id": "4cf77f06-37f8-4a18-8424-08ff79bc6295", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.0969, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:235269bd-32c8-493e-817d-be726e4d5bcc", - "resource": { - "resourceType": "Observation", - "id": "235269bd-32c8-493e-817d-be726e4d5bcc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.1057, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9312514-fdc3-463e-ba7b-3d8ad8505a51", - "resource": { - "resourceType": "Observation", - "id": "c9312514-fdc3-463e-ba7b-3d8ad8505a51", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 1.0496, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b256ec36-7b93-45fa-b918-903299c12c73", - "resource": { - "resourceType": "Observation", - "id": "b256ec36-7b93-45fa-b918-903299c12c73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 75.286, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6153779-6dc8-4b3c-b932-b60afc3ee932", - "resource": { - "resourceType": "Observation", - "id": "b6153779-6dc8-4b3c-b932-b60afc3ee932", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 40.053, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4134cffe-0cc9-4398-aa37-75f505447298", - "resource": { - "resourceType": "Observation", - "id": "4134cffe-0cc9-4398-aa37-75f505447298", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.075, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4eb068e8-009d-418f-adb0-48cce5e82f21", - "resource": { - "resourceType": "Observation", - "id": "4eb068e8-009d-418f-adb0-48cce5e82f21", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.6354, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", - "resource": { - "resourceType": "Observation", - "id": "a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.9082, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c301ef52-3052-459b-89b4-6970154fa4f5", - "resource": { - "resourceType": "Observation", - "id": "c301ef52-3052-459b-89b4-6970154fa4f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.821, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7f38e37a-1be9-4e8a-8298-bca8fee88d50", - "resource": { - "resourceType": "Observation", - "id": "7f38e37a-1be9-4e8a-8298-bca8fee88d50", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 25.788, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a013b221-e582-4ab0-aeb0-78864817cdb2", - "resource": { - "resourceType": "Observation", - "id": "a013b221-e582-4ab0-aeb0-78864817cdb2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 89.699, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0438335c-3f9f-44a3-a4d9-2303a515dbab", - "resource": { - "resourceType": "Observation", - "id": "0438335c-3f9f-44a3-a4d9-2303a515dbab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 31.577, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:038d3409-dc60-4d71-8ec7-f5b6b3940997", - "resource": { - "resourceType": "Observation", - "id": "038d3409-dc60-4d71-8ec7-f5b6b3940997", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 35.19, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c78cb64-c435-4389-8cc4-2d824dad05d3", - "resource": { - "resourceType": "Observation", - "id": "5c78cb64-c435-4389-8cc4-2d824dad05d3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 40.449, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8dbebf8a-c9a9-490b-a5e6-260fb88bd456", - "resource": { - "resourceType": "Observation", - "id": "8dbebf8a-c9a9-490b-a5e6-260fb88bd456", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 249.37, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2e0739b-b9c1-450e-b3cf-d3ab91666e03", - "resource": { - "resourceType": "Observation", - "id": "b2e0739b-b9c1-450e-b3cf-d3ab91666e03", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 175.47, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d75282c-5c83-4c3c-8d57-d803af7cdb70", - "resource": { - "resourceType": "Observation", - "id": "7d75282c-5c83-4c3c-8d57-d803af7cdb70", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 11.843, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3359d124-1545-4f4e-ace9-7160b5137a6f", - "resource": { - "resourceType": "Procedure", - "id": "3359d124-1545-4f4e-ace9-7160b5137a6f", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "performedPeriod": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:30:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:87b9b981-ff42-4eca-8c77-02b964ac4739", - "resource": { - "resourceType": "DiagnosticReport", - "id": "87b9b981-ff42-4eca-8c77-02b964ac4739", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:7d37dc6f-e655-4d03-a64a-bfb7b32c4947", - "display": "Glucose" - }, - { - "reference": "urn:uuid:2372beda-927e-460f-9de0-561b20d4c7a9", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:1c5cf511-b90d-4375-a10d-4b2c38c5e448", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:b8476f78-77fc-41dc-a1e1-482c0eb5a825", - "display": "Calcium" - }, - { - "reference": "urn:uuid:75d3c18e-f2c0-4f63-b7d5-0a1000833542", - "display": "Sodium" - }, - { - "reference": "urn:uuid:71f65388-c251-40b6-bed7-72ce23d07ab5", - "display": "Potassium" - }, - { - "reference": "urn:uuid:f9f36cee-0e35-4c5d-b8e1-f54c66269135", - "display": "Chloride" - }, - { - "reference": "urn:uuid:c0161429-03f9-44c1-9271-2b5f55c7b4bb", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:25fd4b9c-4884-4704-b4c9-f7451c8e0869", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:d981902d-784f-4484-8076-1321f1f5728e", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4cf77f06-37f8-4a18-8424-08ff79bc6295", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:235269bd-32c8-493e-817d-be726e4d5bcc", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:c9312514-fdc3-463e-ba7b-3d8ad8505a51", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:b256ec36-7b93-45fa-b918-903299c12c73", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:b6153779-6dc8-4b3c-b932-b60afc3ee932", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4134cffe-0cc9-4398-aa37-75f505447298", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:30a3cbc5-dec0-4e3d-92d1-93e71f5d7539", - "resource": { - "resourceType": "DiagnosticReport", - "id": "30a3cbc5-dec0-4e3d-92d1-93e71f5d7539", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - }, - "effectiveDateTime": "2013-04-01T05:15:51-07:00", - "issued": "2013-04-01T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:4eb068e8-009d-418f-adb0-48cce5e82f21", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:a0aabfc5-8e77-4a17-be91-b8aecd16f9b5", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:c301ef52-3052-459b-89b4-6970154fa4f5", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:7f38e37a-1be9-4e8a-8298-bca8fee88d50", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:a013b221-e582-4ab0-aeb0-78864817cdb2", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:0438335c-3f9f-44a3-a4d9-2303a515dbab", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:038d3409-dc60-4d71-8ec7-f5b6b3940997", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:5c78cb64-c435-4389-8cc4-2d824dad05d3", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:8dbebf8a-c9a9-490b-a5e6-260fb88bd456", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b2e0739b-b9c1-450e-b3cf-d3ab91666e03", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:7d75282c-5c83-4c3c-8d57-d803af7cdb70", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:5e7abff6-7cd8-4e11-ba75-995e392e8435", - "resource": { - "resourceType": "Claim", - "id": "5e7abff6-7cd8-4e11-ba75-995e392e8435", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:45:51-07:00" - }, - "created": "2013-04-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:3359d124-1545-4f4e-ace9-7160b5137a6f" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 11924.17, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b4f3bdf3-df2f-47f2-ace1-8e47e07ef2b3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b4f3bdf3-df2f-47f2-ace1-8e47e07ef2b3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5e7abff6-7cd8-4e11-ba75-995e392e8435" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-04-01T05:45:51-07:00", - "end": "2014-04-01T05:45:51-07:00" - }, - "created": "2013-04-01T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5e7abff6-7cd8-4e11-ba75-995e392e8435" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:767da137-22c0-4811-8d09-af08dfda2f75" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-04-01T05:15:51-07:00", - "end": "2013-04-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 11924.17, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2384.8340000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9539.336000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11924.17, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11924.17, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 9539.336000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977", - "resource": { - "resourceType": "Encounter", - "id": "7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:45:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3f56b948-7137-4362-96dc-4d481403696d", - "resource": { - "resourceType": "Observation", - "id": "3f56b948-7137-4362-96dc-4d481403696d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 5, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db483d27-9d6b-4a8a-8eae-822d4bc480fa", - "resource": { - "resourceType": "Observation", - "id": "db483d27-9d6b-4a8a-8eae-822d4bc480fa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 91.62, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bdb81812-1403-4cc5-b70f-858f8a50984a", - "resource": { - "resourceType": "Observation", - "id": "bdb81812-1403-4cc5-b70f-858f8a50984a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.28, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:435c85d0-da18-4a27-b2c1-3c43acbffe38", - "resource": { - "resourceType": "Observation", - "id": "435c85d0-da18-4a27-b2c1-3c43acbffe38", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.2633, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c05fead5-b200-4676-8dbb-534a9a88837a", - "resource": { - "resourceType": "Observation", - "id": "c05fead5-b200-4676-8dbb-534a9a88837a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.05, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b28bd42-0541-4b8a-827b-01f90e142e15", - "resource": { - "resourceType": "Observation", - "id": "3b28bd42-0541-4b8a-827b-01f90e142e15", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 138.5, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e0230357-77ff-4a2a-a241-9e6f79d8d8e6", - "resource": { - "resourceType": "Observation", - "id": "e0230357-77ff-4a2a-a241-9e6f79d8d8e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.03, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", - "resource": { - "resourceType": "Observation", - "id": "4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 103.49, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c392cac0-c1cb-4676-8620-bddfa9e59dff", - "resource": { - "resourceType": "Observation", - "id": "c392cac0-c1cb-4676-8620-bddfa9e59dff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 25.95, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:890c6989-1e4a-4fe4-9ab5-cb1095d5437d", - "resource": { - "resourceType": "Observation", - "id": "890c6989-1e4a-4fe4-9ab5-cb1095d5437d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 14.388, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7562b88d-3f43-4089-95af-bf82c68f31d8", - "resource": { - "resourceType": "Observation", - "id": "7562b88d-3f43-4089-95af-bf82c68f31d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 69.584, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c440773-3dff-4822-a315-46fc30f099c6", - "resource": { - "resourceType": "Observation", - "id": "4c440773-3dff-4822-a315-46fc30f099c6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.4058, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", - "resource": { - "resourceType": "Observation", - "id": "4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.4536, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cdde0e90-236d-4d49-b6b6-64c010a9b118", - "resource": { - "resourceType": "Observation", - "id": "cdde0e90-236d-4d49-b6b6-64c010a9b118", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.76234, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0213fc21-7fde-4d6b-9043-0231c3f8eac5", - "resource": { - "resourceType": "Observation", - "id": "0213fc21-7fde-4d6b-9043-0231c3f8eac5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 50.534, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f0691777-00ab-49ce-b56d-ada1036eeab5", - "resource": { - "resourceType": "Observation", - "id": "f0691777-00ab-49ce-b56d-ada1036eeab5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 57.686, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:199a0c22-29cf-4e5d-9d05-3ba361fd8687", - "resource": { - "resourceType": "Observation", - "id": "199a0c22-29cf-4e5d-9d05-3ba361fd8687", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 32.071, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5654fec6-cf82-4e31-afcf-a53507f24661", - "resource": { - "resourceType": "Observation", - "id": "5654fec6-cf82-4e31-afcf-a53507f24661", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.0567, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", - "resource": { - "resourceType": "Observation", - "id": "12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.9416, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb6f8e83-79cf-4508-ba06-a156a966eb8a", - "resource": { - "resourceType": "Observation", - "id": "fb6f8e83-79cf-4508-ba06-a156a966eb8a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 12.142, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba58aa20-515d-41e7-a436-c48af6109029", - "resource": { - "resourceType": "Observation", - "id": "ba58aa20-515d-41e7-a436-c48af6109029", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 25.632, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", - "resource": { - "resourceType": "Observation", - "id": "03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 86.208, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a92e16af-1756-4719-be21-9cece29bc38a", - "resource": { - "resourceType": "Observation", - "id": "a92e16af-1756-4719-be21-9cece29bc38a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 31.819, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c14633d-fd29-4ec5-9504-1c816edb6380", - "resource": { - "resourceType": "Observation", - "id": "3c14633d-fd29-4ec5-9504-1c816edb6380", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 34.785, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e075aabd-4c38-4a01-811f-b632c5485b67", - "resource": { - "resourceType": "Observation", - "id": "e075aabd-4c38-4a01-811f-b632c5485b67", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 43.731, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:684b6ca4-14a0-4b42-8c62-e22ad914963a", - "resource": { - "resourceType": "Observation", - "id": "684b6ca4-14a0-4b42-8c62-e22ad914963a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 447.75, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", - "resource": { - "resourceType": "Observation", - "id": "5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 408.89, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d25ac0a-ec16-489c-90aa-636271a2fdf1", - "resource": { - "resourceType": "Observation", - "id": "5d25ac0a-ec16-489c-90aa-636271a2fdf1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 11.461, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:88d2e17e-e828-420d-b9c8-7dc475aa93a4", - "resource": { - "resourceType": "Procedure", - "id": "88d2e17e-e828-420d-b9c8-7dc475aa93a4", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "performedPeriod": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:30:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d298345a-915e-41d5-9ba6-a8cac8e028fc", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d298345a-915e-41d5-9ba6-a8cac8e028fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:db483d27-9d6b-4a8a-8eae-822d4bc480fa", - "display": "Glucose" - }, - { - "reference": "urn:uuid:bdb81812-1403-4cc5-b70f-858f8a50984a", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:435c85d0-da18-4a27-b2c1-3c43acbffe38", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:c05fead5-b200-4676-8dbb-534a9a88837a", - "display": "Calcium" - }, - { - "reference": "urn:uuid:3b28bd42-0541-4b8a-827b-01f90e142e15", - "display": "Sodium" - }, - { - "reference": "urn:uuid:e0230357-77ff-4a2a-a241-9e6f79d8d8e6", - "display": "Potassium" - }, - { - "reference": "urn:uuid:4a7ca3f3-bf43-4dbd-8484-2de3cc21c47e", - "display": "Chloride" - }, - { - "reference": "urn:uuid:c392cac0-c1cb-4676-8620-bddfa9e59dff", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:890c6989-1e4a-4fe4-9ab5-cb1095d5437d", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:7562b88d-3f43-4089-95af-bf82c68f31d8", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4c440773-3dff-4822-a315-46fc30f099c6", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4f6eccd3-c30c-4f2b-98b0-c75b855d4ab9", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:cdde0e90-236d-4d49-b6b6-64c010a9b118", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:0213fc21-7fde-4d6b-9043-0231c3f8eac5", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:f0691777-00ab-49ce-b56d-ada1036eeab5", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:199a0c22-29cf-4e5d-9d05-3ba361fd8687", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:4223c0c9-07d7-4f8b-bd12-4276cc12f3c2", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4223c0c9-07d7-4f8b-bd12-4276cc12f3c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - }, - "effectiveDateTime": "2013-05-03T05:15:51-07:00", - "issued": "2013-05-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:5654fec6-cf82-4e31-afcf-a53507f24661", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:12d76f63-23ea-40b4-b7ae-4ffc0cee45e9", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:fb6f8e83-79cf-4508-ba06-a156a966eb8a", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:ba58aa20-515d-41e7-a436-c48af6109029", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:03a60cbd-8fd2-48af-9e76-67a1ed44f9f6", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:a92e16af-1756-4719-be21-9cece29bc38a", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:3c14633d-fd29-4ec5-9504-1c816edb6380", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:e075aabd-4c38-4a01-811f-b632c5485b67", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:684b6ca4-14a0-4b42-8c62-e22ad914963a", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:5e462d83-a9f1-4e9b-ad03-19472b9d3bd3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:5d25ac0a-ec16-489c-90aa-636271a2fdf1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:567a83b9-9037-4a16-b72c-008a71ece928", - "resource": { - "resourceType": "Claim", - "id": "567a83b9-9037-4a16-b72c-008a71ece928", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:45:51-07:00" - }, - "created": "2013-05-03T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:88d2e17e-e828-420d-b9c8-7dc475aa93a4" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 13022.01, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:292ea288-6a0f-4352-88b0-a7ce25abcb16", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "292ea288-6a0f-4352-88b0-a7ce25abcb16", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "567a83b9-9037-4a16-b72c-008a71ece928" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-05-03T05:45:51-07:00", - "end": "2014-05-03T05:45:51-07:00" - }, - "created": "2013-05-03T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:567a83b9-9037-4a16-b72c-008a71ece928" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7f2630d5-5eb3-4a0c-9a32-8fb5afc4c977" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-05-03T05:15:51-07:00", - "end": "2013-05-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 13022.01, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2604.402, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 10417.608, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 13022.01, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 13022.01, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 10417.608, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266", - "resource": { - "resourceType": "Encounter", - "id": "25007a07-a29f-45f1-b210-46ba67fc0266", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:45:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f6f32e83-b399-4769-9e79-7b8d875ca96c", - "resource": { - "resourceType": "Observation", - "id": "f6f32e83-b399-4769-9e79-7b8d875ca96c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 8, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e10e3f7f-a4a4-40e5-b949-508a04d5a989", - "resource": { - "resourceType": "Observation", - "id": "e10e3f7f-a4a4-40e5-b949-508a04d5a989", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 89.41, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3be48981-0679-4de8-9911-ec13747379df", - "resource": { - "resourceType": "Observation", - "id": "3be48981-0679-4de8-9911-ec13747379df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 13.41, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e7da76c-9d45-431d-8551-28d544f3a66e", - "resource": { - "resourceType": "Observation", - "id": "1e7da76c-9d45-431d-8551-28d544f3a66e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.9915, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d0502848-2679-4d4b-8501-26ad0eecf9eb", - "resource": { - "resourceType": "Observation", - "id": "d0502848-2679-4d4b-8501-26ad0eecf9eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 8.56, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bd5e6353-45ac-4751-93af-beeb8255958c", - "resource": { - "resourceType": "Observation", - "id": "bd5e6353-45ac-4751-93af-beeb8255958c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 137.87, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", - "resource": { - "resourceType": "Observation", - "id": "b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.74, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", - "resource": { - "resourceType": "Observation", - "id": "fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 102.86, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", - "resource": { - "resourceType": "Observation", - "id": "7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 25.44, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b1279371-8e99-41b2-b7ff-5b3a97efe602", - "resource": { - "resourceType": "Observation", - "id": "b1279371-8e99-41b2-b7ff-5b3a97efe602", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.416, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:de5417cb-d88a-4982-9f74-b3d6e9ccb723", - "resource": { - "resourceType": "Observation", - "id": "de5417cb-d88a-4982-9f74-b3d6e9ccb723", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 76.97, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2f31c342-3e04-4ce2-b98e-2c27809c2863", - "resource": { - "resourceType": "Observation", - "id": "2f31c342-3e04-4ce2-b98e-2c27809c2863", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.2324, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7a09bece-90ba-43eb-8cbd-55a8655cb351", - "resource": { - "resourceType": "Observation", - "id": "7a09bece-90ba-43eb-8cbd-55a8655cb351", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.2795, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee676515-2265-47af-9a03-dbb01b7fa1e6", - "resource": { - "resourceType": "Observation", - "id": "ee676515-2265-47af-9a03-dbb01b7fa1e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.73116, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", - "resource": { - "resourceType": "Observation", - "id": "ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 94.61, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4006caea-a194-4c2b-a6bc-0e978cb80011", - "resource": { - "resourceType": "Observation", - "id": "4006caea-a194-4c2b-a6bc-0e978cb80011", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 28.356, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d341f336-3669-427b-ba4c-698784affc7c", - "resource": { - "resourceType": "Observation", - "id": "d341f336-3669-427b-ba4c-698784affc7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 19.842, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e9974a8-1f57-47ae-9d0d-90de9bd6b553", - "resource": { - "resourceType": "Observation", - "id": "1e9974a8-1f57-47ae-9d0d-90de9bd6b553", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 1.1343, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:376bdd74-8b7c-4c1e-865f-67c9db93c527", - "resource": { - "resourceType": "Observation", - "id": "376bdd74-8b7c-4c1e-865f-67c9db93c527", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.4919, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b44f0182-af4c-4722-9bee-e48a3458fc74", - "resource": { - "resourceType": "Observation", - "id": "b44f0182-af4c-4722-9bee-e48a3458fc74", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.033, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", - "resource": { - "resourceType": "Observation", - "id": "5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 30.417, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c383a21-1bdd-4b14-afb3-7328aefeb7c5", - "resource": { - "resourceType": "Observation", - "id": "2c383a21-1bdd-4b14-afb3-7328aefeb7c5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 82.847, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", - "resource": { - "resourceType": "Observation", - "id": "f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 29.018, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:13390f28-fb82-4110-9c9f-99a2d71adf98", - "resource": { - "resourceType": "Observation", - "id": "13390f28-fb82-4110-9c9f-99a2d71adf98", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 33.752, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", - "resource": { - "resourceType": "Observation", - "id": "7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 44.993, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:89afeeab-ffd7-4a94-a6cf-a65c81fc4727", - "resource": { - "resourceType": "Observation", - "id": "89afeeab-ffd7-4a94-a6cf-a65c81fc4727", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 161.69, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3fc4c1ed-91fa-415c-97b6-6775961ab826", - "resource": { - "resourceType": "Observation", - "id": "3fc4c1ed-91fa-415c-97b6-6775961ab826", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 184.06, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d0224e91-ce6e-499b-a927-e157b3623ff0", - "resource": { - "resourceType": "Observation", - "id": "d0224e91-ce6e-499b-a927-e157b3623ff0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 10.339, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8df22fb6-c5ee-48ed-86e0-f76f19798fb7", - "resource": { - "resourceType": "Procedure", - "id": "8df22fb6-c5ee-48ed-86e0-f76f19798fb7", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "performedPeriod": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:30:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9c166ad3-9123-4bd6-8140-d3749b2c2bae", - "resource": { - "resourceType": "DiagnosticReport", - "id": "9c166ad3-9123-4bd6-8140-d3749b2c2bae", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:e10e3f7f-a4a4-40e5-b949-508a04d5a989", - "display": "Glucose" - }, - { - "reference": "urn:uuid:3be48981-0679-4de8-9911-ec13747379df", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:1e7da76c-9d45-431d-8551-28d544f3a66e", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:d0502848-2679-4d4b-8501-26ad0eecf9eb", - "display": "Calcium" - }, - { - "reference": "urn:uuid:bd5e6353-45ac-4751-93af-beeb8255958c", - "display": "Sodium" - }, - { - "reference": "urn:uuid:b6ae9f1b-cf93-4259-8d87-2084e7a82ea6", - "display": "Potassium" - }, - { - "reference": "urn:uuid:fd6c4814-4d8c-4a10-8e8f-4aa51876e1cb", - "display": "Chloride" - }, - { - "reference": "urn:uuid:7e6bb8b5-4dcc-40fc-bf24-ba5e86b65457", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:b1279371-8e99-41b2-b7ff-5b3a97efe602", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:de5417cb-d88a-4982-9f74-b3d6e9ccb723", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:2f31c342-3e04-4ce2-b98e-2c27809c2863", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:7a09bece-90ba-43eb-8cbd-55a8655cb351", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:ee676515-2265-47af-9a03-dbb01b7fa1e6", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:ec58a0b9-d8fb-48a8-9b7c-cbe4b703226a", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4006caea-a194-4c2b-a6bc-0e978cb80011", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:d341f336-3669-427b-ba4c-698784affc7c", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:d5a0caa4-f0a5-488f-a1fb-75c0b2e5120f", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d5a0caa4-f0a5-488f-a1fb-75c0b2e5120f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - }, - "effectiveDateTime": "2013-06-03T05:15:51-07:00", - "issued": "2013-06-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:1e9974a8-1f57-47ae-9d0d-90de9bd6b553", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:376bdd74-8b7c-4c1e-865f-67c9db93c527", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b44f0182-af4c-4722-9bee-e48a3458fc74", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:5489f6bb-0336-4ab8-ab9d-7bb7d1ed8448", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:2c383a21-1bdd-4b14-afb3-7328aefeb7c5", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:f031a2b2-05f5-47f2-8bfe-832ccdaa29c8", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:13390f28-fb82-4110-9c9f-99a2d71adf98", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:7ae080e5-3b3c-40af-a4b8-17dea0b4f1a9", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:89afeeab-ffd7-4a94-a6cf-a65c81fc4727", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:3fc4c1ed-91fa-415c-97b6-6775961ab826", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:d0224e91-ce6e-499b-a927-e157b3623ff0", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa", - "resource": { - "resourceType": "Claim", - "id": "6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:45:51-07:00" - }, - "created": "2013-06-03T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:8df22fb6-c5ee-48ed-86e0-f76f19798fb7" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 13305.82, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:84114e2d-ea86-4f2d-9f76-e8fdc2a83693", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "84114e2d-ea86-4f2d-9f76-e8fdc2a83693", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-06-03T05:45:51-07:00", - "end": "2014-06-03T05:45:51-07:00" - }, - "created": "2013-06-03T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:6e1437dc-a7c5-4eb0-9624-c07d9fcd51aa" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:25007a07-a29f-45f1-b210-46ba67fc0266" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-06-03T05:15:51-07:00", - "end": "2013-06-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 13305.82, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2661.164, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 10644.656, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 13305.82, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 13305.82, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 10644.656, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320", - "resource": { - "resourceType": "Encounter", - "id": "7009e93b-a80f-4c2e-98c0-450dcb2ee320", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:45:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d4fb7920-826e-412f-a0b2-434698867a36", - "resource": { - "resourceType": "Observation", - "id": "d4fb7920-826e-412f-a0b2-434698867a36", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 6, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a4ea7bc1-0ace-484f-8fff-d447cca6a288", - "resource": { - "resourceType": "Observation", - "id": "a4ea7bc1-0ace-484f-8fff-d447cca6a288", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 93.35, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50263b45-98e1-4f66-96ea-b99009163f88", - "resource": { - "resourceType": "Observation", - "id": "50263b45-98e1-4f66-96ea-b99009163f88", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 12.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed4b9588-f6d7-4a39-9aef-dcc51d087055", - "resource": { - "resourceType": "Observation", - "id": "ed4b9588-f6d7-4a39-9aef-dcc51d087055", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.9351, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4db612bc-9da6-4d3d-b7db-fff64e0df3e9", - "resource": { - "resourceType": "Observation", - "id": "4db612bc-9da6-4d3d-b7db-fff64e0df3e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.82, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:055632d4-9655-4066-aa4f-e7bc8962c5cc", - "resource": { - "resourceType": "Observation", - "id": "055632d4-9655-4066-aa4f-e7bc8962c5cc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 141.37, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6bbb3f02-95df-40b3-8a9c-d4dd59246994", - "resource": { - "resourceType": "Observation", - "id": "6bbb3f02-95df-40b3-8a9c-d4dd59246994", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.14, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9abe6e75-09f1-44f1-b141-7aa01c9c54e0", - "resource": { - "resourceType": "Observation", - "id": "9abe6e75-09f1-44f1-b141-7aa01c9c54e0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 102.1, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11c98316-11f0-4215-ba70-ae58507353b1", - "resource": { - "resourceType": "Observation", - "id": "11c98316-11f0-4215-ba70-ae58507353b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 23.18, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c8b1ccb1-c19b-40a3-af13-3a54e6e06397", - "resource": { - "resourceType": "Observation", - "id": "c8b1ccb1-c19b-40a3-af13-3a54e6e06397", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 7.9529, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", - "resource": { - "resourceType": "Observation", - "id": "01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 76.827, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:91975c4e-4ddb-4774-a317-35cb425ae7eb", - "resource": { - "resourceType": "Observation", - "id": "91975c4e-4ddb-4774-a317-35cb425ae7eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.9537, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e16012c-f25a-4ccd-bcbc-3f8064f5284d", - "resource": { - "resourceType": "Observation", - "id": "4e16012c-f25a-4ccd-bcbc-3f8064f5284d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.9078, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5901e8e5-a877-4545-afa9-963dea832a63", - "resource": { - "resourceType": "Observation", - "id": "5901e8e5-a877-4545-afa9-963dea832a63", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.29781, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:da2381dd-5766-4a49-b30c-3c4f81dd4729", - "resource": { - "resourceType": "Observation", - "id": "da2381dd-5766-4a49-b30c-3c4f81dd4729", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 56.614, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", - "resource": { - "resourceType": "Observation", - "id": "4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 57.996, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:73e22621-70c6-4862-a5e7-2ee5c133b1aa", - "resource": { - "resourceType": "Observation", - "id": "73e22621-70c6-4862-a5e7-2ee5c133b1aa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 26.657, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2f7908cb-b33d-409f-a209-6c42835c6fe4", - "resource": { - "resourceType": "Observation", - "id": "2f7908cb-b33d-409f-a209-6c42835c6fe4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.6457, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cdcc9f05-f61a-4a96-8f91-4cc18e15758d", - "resource": { - "resourceType": "Observation", - "id": "cdcc9f05-f61a-4a96-8f91-4cc18e15758d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 5.4292, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c44df0f8-e1d7-4ec7-9a53-a5c78616f455", - "resource": { - "resourceType": "Observation", - "id": "c44df0f8-e1d7-4ec7-9a53-a5c78616f455", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 12.151, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:08bebef2-cab9-4d3c-989b-bdc5c6719a79", - "resource": { - "resourceType": "Observation", - "id": "08bebef2-cab9-4d3c-989b-bdc5c6719a79", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 32.333, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0257b1a-b770-4d89-9221-ead5e073ed09", - "resource": { - "resourceType": "Observation", - "id": "b0257b1a-b770-4d89-9221-ead5e073ed09", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 92.156, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1234710d-7995-4d8b-a6c6-f30354d3ef15", - "resource": { - "resourceType": "Observation", - "id": "1234710d-7995-4d8b-a6c6-f30354d3ef15", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 29.12, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9d2361d-756a-4fef-8b1e-c570d04f8e12", - "resource": { - "resourceType": "Observation", - "id": "a9d2361d-756a-4fef-8b1e-c570d04f8e12", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 34.547, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3ea2b6fc-982b-464d-8b7d-1c3c124c5192", - "resource": { - "resourceType": "Observation", - "id": "3ea2b6fc-982b-464d-8b7d-1c3c124c5192", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 40.081, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4f51dfab-e10b-40fb-ba72-993732dd716c", - "resource": { - "resourceType": "Observation", - "id": "4f51dfab-e10b-40fb-ba72-993732dd716c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 196.26, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c65c3db-8493-4dba-a427-8ccbd3d01aa4", - "resource": { - "resourceType": "Observation", - "id": "1c65c3db-8493-4dba-a427-8ccbd3d01aa4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 381.15, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:89d4bb3b-63df-45ce-9997-facc4934fb49", - "resource": { - "resourceType": "Observation", - "id": "89d4bb3b-63df-45ce-9997-facc4934fb49", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "valueQuantity": { - "value": 12.3, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8301b201-36f6-4076-ad21-33fb2e23fe95", - "resource": { - "resourceType": "Procedure", - "id": "8301b201-36f6-4076-ad21-33fb2e23fe95", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "performedPeriod": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:30:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d7d87227-51f3-4ac0-82f8-1d4f1daf5346", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d7d87227-51f3-4ac0-82f8-1d4f1daf5346", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:a4ea7bc1-0ace-484f-8fff-d447cca6a288", - "display": "Glucose" - }, - { - "reference": "urn:uuid:50263b45-98e1-4f66-96ea-b99009163f88", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:ed4b9588-f6d7-4a39-9aef-dcc51d087055", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:4db612bc-9da6-4d3d-b7db-fff64e0df3e9", - "display": "Calcium" - }, - { - "reference": "urn:uuid:055632d4-9655-4066-aa4f-e7bc8962c5cc", - "display": "Sodium" - }, - { - "reference": "urn:uuid:6bbb3f02-95df-40b3-8a9c-d4dd59246994", - "display": "Potassium" - }, - { - "reference": "urn:uuid:9abe6e75-09f1-44f1-b141-7aa01c9c54e0", - "display": "Chloride" - }, - { - "reference": "urn:uuid:11c98316-11f0-4215-ba70-ae58507353b1", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:c8b1ccb1-c19b-40a3-af13-3a54e6e06397", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:01c3f8b0-1cca-4a33-b381-9db2fd47bf6e", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:91975c4e-4ddb-4774-a317-35cb425ae7eb", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4e16012c-f25a-4ccd-bcbc-3f8064f5284d", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:5901e8e5-a877-4545-afa9-963dea832a63", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:da2381dd-5766-4a49-b30c-3c4f81dd4729", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4b5f0ced-cf94-4ee9-880f-7283a4eb51f6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:73e22621-70c6-4862-a5e7-2ee5c133b1aa", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f199e05c-8fca-424d-8725-d1e23528b1e0", - "resource": { - "resourceType": "DiagnosticReport", - "id": "f199e05c-8fca-424d-8725-d1e23528b1e0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - }, - "effectiveDateTime": "2013-07-01T05:15:51-07:00", - "issued": "2013-07-01T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:2f7908cb-b33d-409f-a209-6c42835c6fe4", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:cdcc9f05-f61a-4a96-8f91-4cc18e15758d", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:c44df0f8-e1d7-4ec7-9a53-a5c78616f455", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:08bebef2-cab9-4d3c-989b-bdc5c6719a79", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:b0257b1a-b770-4d89-9221-ead5e073ed09", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:1234710d-7995-4d8b-a6c6-f30354d3ef15", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:a9d2361d-756a-4fef-8b1e-c570d04f8e12", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:3ea2b6fc-982b-464d-8b7d-1c3c124c5192", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:4f51dfab-e10b-40fb-ba72-993732dd716c", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1c65c3db-8493-4dba-a427-8ccbd3d01aa4", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:89d4bb3b-63df-45ce-9997-facc4934fb49", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0ec20549-3f16-4ae2-ba60-569c3f80ad65", - "resource": { - "resourceType": "Claim", - "id": "0ec20549-3f16-4ae2-ba60-569c3f80ad65", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:45:51-07:00" - }, - "created": "2013-07-01T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:8301b201-36f6-4076-ad21-33fb2e23fe95" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 9589.76, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6fc952e0-1cb7-4b84-ba46-22ce2056e59b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6fc952e0-1cb7-4b84-ba46-22ce2056e59b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0ec20549-3f16-4ae2-ba60-569c3f80ad65" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-07-01T05:45:51-07:00", - "end": "2014-07-01T05:45:51-07:00" - }, - "created": "2013-07-01T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0ec20549-3f16-4ae2-ba60-569c3f80ad65" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7009e93b-a80f-4c2e-98c0-450dcb2ee320" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-07-01T05:15:51-07:00", - "end": "2013-07-01T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 9589.76, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1917.9520000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7671.808000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 9589.76, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 9589.76, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7671.808000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69", - "resource": { - "resourceType": "Encounter", - "id": "ee0032a1-183c-4106-a058-b530e2e8df69", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:45:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:45:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c1af820a-72e4-4b68-8a08-67674f95156b", - "resource": { - "resourceType": "Observation", - "id": "c1af820a-72e4-4b68-8a08-67674f95156b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 7, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3668a3c-b125-4c1f-bcb9-2718c969bda1", - "resource": { - "resourceType": "Observation", - "id": "e3668a3c-b125-4c1f-bcb9-2718c969bda1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 86.96, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3100271-6dab-4757-8162-33aa845b2be1", - "resource": { - "resourceType": "Observation", - "id": "e3100271-6dab-4757-8162-33aa845b2be1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 15.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9e47b447-6e7e-464c-9130-8a9459f67335", - "resource": { - "resourceType": "Observation", - "id": "9e47b447-6e7e-464c-9130-8a9459f67335", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.5959, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0275af97-f3dc-44ac-a196-a0427c5130f4", - "resource": { - "resourceType": "Observation", - "id": "0275af97-f3dc-44ac-a196-a0427c5130f4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 9.32, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7ac76bed-dd47-4e2d-bb84-3b4e884d3277", - "resource": { - "resourceType": "Observation", - "id": "7ac76bed-dd47-4e2d-bb84-3b4e884d3277", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 139.22, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:594abc12-1f5d-4514-b14f-f55fec4a1cc2", - "resource": { - "resourceType": "Observation", - "id": "594abc12-1f5d-4514-b14f-f55fec4a1cc2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.65, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c712bc3-0b86-4b38-bfa8-5696add93de2", - "resource": { - "resourceType": "Observation", - "id": "0c712bc3-0b86-4b38-bfa8-5696add93de2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 108.64, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34afb9aa-f874-493b-b286-4319b1e1f950", - "resource": { - "resourceType": "Observation", - "id": "34afb9aa-f874-493b-b286-4319b1e1f950", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 25.89, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bf9e0a34-aa41-4d98-b69a-4d9a3762e349", - "resource": { - "resourceType": "Observation", - "id": "bf9e0a34-aa41-4d98-b69a-4d9a3762e349", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33914-3", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - } - ], - "text": "Glomerular filtration rate/1.73 sq M.predicted" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 23.513, - "unit": "mL/min", - "system": "http://unitsofmeasure.org", - "code": "mL/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2f5ed8d-9778-4fe6-a38a-0d08becb519a", - "resource": { - "resourceType": "Observation", - "id": "e2f5ed8d-9778-4fe6-a38a-0d08becb519a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2885-2", - "display": "Protein [Mass/volume] in Serum or Plasma" - } - ], - "text": "Protein [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 71.487, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", - "resource": { - "resourceType": "Observation", - "id": "4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1751-7", - "display": "Albumin [Mass/volume] in Serum or Plasma" - } - ], - "text": "Albumin [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 3.5381, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:950b45e8-69e9-4a4e-b934-38790b46e387", - "resource": { - "resourceType": "Observation", - "id": "950b45e8-69e9-4a4e-b934-38790b46e387", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "10834-0", - "display": "Globulin [Mass/volume] in Serum by calculation" - } - ], - "text": "Globulin [Mass/volume] in Serum by calculation" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.8078, - "unit": "g/L", - "system": "http://unitsofmeasure.org", - "code": "g/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:90d66f1a-7eaf-40b8-ac33-b3a65956ec11", - "resource": { - "resourceType": "Observation", - "id": "90d66f1a-7eaf-40b8-ac33-b3a65956ec11", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1975-2", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - } - ], - "text": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 0.3802, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:901f8064-d960-444f-acc8-020b767f39f6", - "resource": { - "resourceType": "Observation", - "id": "901f8064-d960-444f-acc8-020b767f39f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6768-6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 136.15, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25336e04-d01b-4aa8-9ee9-2daed53b3380", - "resource": { - "resourceType": "Observation", - "id": "25336e04-d01b-4aa8-9ee9-2daed53b3380", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1742-6", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 20.644, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:57c183ce-8062-4059-8fb9-230cbb45dace", - "resource": { - "resourceType": "Observation", - "id": "57c183ce-8062-4059-8fb9-230cbb45dace", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "1920-8", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ], - "text": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 14.904, - "unit": "U/L", - "system": "http://unitsofmeasure.org", - "code": "U/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4a1bfb0-21f1-4731-b150-56287fe9e8d2", - "resource": { - "resourceType": "Observation", - "id": "d4a1bfb0-21f1-4731-b150-56287fe9e8d2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 2.483, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e919c3e-abd3-4f1b-8d19-51b6b305eda7", - "resource": { - "resourceType": "Observation", - "id": "7e919c3e-abd3-4f1b-8d19-51b6b305eda7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 4.4587, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c86cf3ba-5cc2-4987-9525-25d7cb3960ec", - "resource": { - "resourceType": "Observation", - "id": "c86cf3ba-5cc2-4987-9525-25d7cb3960ec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 11.79, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:61582809-9b25-4cf7-9601-c052baefa448", - "resource": { - "resourceType": "Observation", - "id": "61582809-9b25-4cf7-9601-c052baefa448", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 30.912, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:637b3bc4-a0d4-4991-bae6-d0844683d041", - "resource": { - "resourceType": "Observation", - "id": "637b3bc4-a0d4-4991-bae6-d0844683d041", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 88.162, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ea779f6a-ea11-4bb0-a479-8949b497b137", - "resource": { - "resourceType": "Observation", - "id": "ea779f6a-ea11-4bb0-a479-8949b497b137", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 30.265, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0beca38-819a-4d66-b2e1-91f0dee928e0", - "resource": { - "resourceType": "Observation", - "id": "c0beca38-819a-4d66-b2e1-91f0dee928e0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 34.142, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:39678dda-38d6-44d2-b4c0-b187d8054845", - "resource": { - "resourceType": "Observation", - "id": "39678dda-38d6-44d2-b4c0-b187d8054845", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 41.58, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e14abdb0-87de-425f-b84a-6921fd094428", - "resource": { - "resourceType": "Observation", - "id": "e14abdb0-87de-425f-b84a-6921fd094428", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 188.57, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:88868082-4d03-43ff-b9a8-c20fd8a3d807", - "resource": { - "resourceType": "Observation", - "id": "88868082-4d03-43ff-b9a8-c20fd8a3d807", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 357.31, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cc38aa2-262d-40f2-bb59-fc288edfd0fe", - "resource": { - "resourceType": "Observation", - "id": "0cc38aa2-262d-40f2-bb59-fc288edfd0fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "valueQuantity": { - "value": 11.371, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4fee3798-31b4-4cb9-8bc7-1e8118c962c4", - "resource": { - "resourceType": "Procedure", - "id": "4fee3798-31b4-4cb9-8bc7-1e8118c962c4", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "performedPeriod": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:30:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:a379ccbd-e836-4098-bb4d-623c0b4f5cd4", - "display": "Overlapping malignant neoplasm of colon" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e576d614-822a-43b9-b66a-993e451bee8c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "e576d614-822a-43b9-b66a-993e451bee8c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "24323-8", - "display": "Comprehensive metabolic 2000 panel - Serum or Plasma" - } - ], - "text": "Comprehensive metabolic 2000 panel - Serum or Plasma" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:e3668a3c-b125-4c1f-bcb9-2718c969bda1", - "display": "Glucose" - }, - { - "reference": "urn:uuid:e3100271-6dab-4757-8162-33aa845b2be1", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:9e47b447-6e7e-464c-9130-8a9459f67335", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:0275af97-f3dc-44ac-a196-a0427c5130f4", - "display": "Calcium" - }, - { - "reference": "urn:uuid:7ac76bed-dd47-4e2d-bb84-3b4e884d3277", - "display": "Sodium" - }, - { - "reference": "urn:uuid:594abc12-1f5d-4514-b14f-f55fec4a1cc2", - "display": "Potassium" - }, - { - "reference": "urn:uuid:0c712bc3-0b86-4b38-bfa8-5696add93de2", - "display": "Chloride" - }, - { - "reference": "urn:uuid:34afb9aa-f874-493b-b286-4319b1e1f950", - "display": "Carbon Dioxide" - }, - { - "reference": "urn:uuid:bf9e0a34-aa41-4d98-b69a-4d9a3762e349", - "display": "Glomerular filtration rate/1.73 sq M.predicted" - }, - { - "reference": "urn:uuid:e2f5ed8d-9778-4fe6-a38a-0d08becb519a", - "display": "Protein [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:4fd9a570-0c64-4bd3-a7cb-6fca36e5d449", - "display": "Albumin [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:950b45e8-69e9-4a4e-b934-38790b46e387", - "display": "Globulin [Mass/volume] in Serum by calculation" - }, - { - "reference": "urn:uuid:90d66f1a-7eaf-40b8-ac33-b3a65956ec11", - "display": "Bilirubin.total [Mass/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:901f8064-d960-444f-acc8-020b767f39f6", - "display": "Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:25336e04-d01b-4aa8-9ee9-2daed53b3380", - "display": "Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - }, - { - "reference": "urn:uuid:57c183ce-8062-4059-8fb9-230cbb45dace", - "display": "Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:83d2fa1e-edd6-47d5-99e8-1571d1f39f4c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "83d2fa1e-edd6-47d5-99e8-1571d1f39f4c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - }, - "effectiveDateTime": "2013-08-03T05:15:51-07:00", - "issued": "2013-08-03T05:15:51.927-07:00", - "result": [ - { - "reference": "urn:uuid:d4a1bfb0-21f1-4731-b150-56287fe9e8d2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:7e919c3e-abd3-4f1b-8d19-51b6b305eda7", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:c86cf3ba-5cc2-4987-9525-25d7cb3960ec", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:61582809-9b25-4cf7-9601-c052baefa448", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:637b3bc4-a0d4-4991-bae6-d0844683d041", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:ea779f6a-ea11-4bb0-a479-8949b497b137", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:c0beca38-819a-4d66-b2e1-91f0dee928e0", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:39678dda-38d6-44d2-b4c0-b187d8054845", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:e14abdb0-87de-425f-b84a-6921fd094428", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:88868082-4d03-43ff-b9a8-c20fd8a3d807", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:0cc38aa2-262d-40f2-bb59-fc288edfd0fe", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030", - "resource": { - "resourceType": "Claim", - "id": "b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:45:51-07:00" - }, - "created": "2013-08-03T05:45:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4fee3798-31b4-4cb9-8bc7-1e8118c962c4" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "net": { - "value": 16455.78, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3add92b6-2b88-42c3-9f6c-3fd5dbe37a04", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3add92b6-2b88-42c3-9f6c-3fd5dbe37a04", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-08-03T05:45:51-07:00", - "end": "2014-08-03T05:45:51-07:00" - }, - "created": "2013-08-03T05:45:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b9953b5b-c72f-4c8b-8fcd-1f8e6a89d030" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ee0032a1-183c-4106-a058-b530e2e8df69" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703423002", - "display": "Combined chemotherapy and radiation therapy (procedure)" - } - ], - "text": "Combined chemotherapy and radiation therapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-08-03T05:15:51-07:00", - "end": "2013-08-03T05:45:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 16455.78, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 3291.156, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 13164.624, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 16455.78, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 16455.78, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 13164.624, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455", - "resource": { - "resourceType": "Encounter", - "id": "5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:64ffad44-b085-4412-a9be-f775e0786881", - "resource": { - "resourceType": "Observation", - "id": "64ffad44-b085-4412-a9be-f775e0786881", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d9b8422-c3c6-4a8c-9853-ee19f0565a33", - "resource": { - "resourceType": "Observation", - "id": "3d9b8422-c3c6-4a8c-9853-ee19f0565a33", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:54b8328f-9ab2-4852-998a-6a0853292537", - "resource": { - "resourceType": "Observation", - "id": "54b8328f-9ab2-4852-998a-6a0853292537", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:081942d4-b592-411c-9e0b-d00b317de99b", - "resource": { - "resourceType": "Observation", - "id": "081942d4-b592-411c-9e0b-d00b317de99b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e878ed4-644f-449c-91c0-a69df02bb0eb", - "resource": { - "resourceType": "Observation", - "id": "6e878ed4-644f-449c-91c0-a69df02bb0eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 122, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa1ffeab-e59f-47ea-b09e-1848ab36656b", - "resource": { - "resourceType": "Observation", - "id": "fa1ffeab-e59f-47ea-b09e-1848ab36656b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 65, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c11ca22-4d01-44f5-9e63-0f77ccc96790", - "resource": { - "resourceType": "Observation", - "id": "0c11ca22-4d01-44f5-9e63-0f77ccc96790", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bd4b198f-5e03-44ce-978c-ee474501fc5f", - "resource": { - "resourceType": "Observation", - "id": "bd4b198f-5e03-44ce-978c-ee474501fc5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 72.76, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:01c7a467-8130-4c85-9a6a-c5cd25140ccb", - "resource": { - "resourceType": "Observation", - "id": "01c7a467-8130-4c85-9a6a-c5cd25140ccb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 10.73, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9abdda58-25c5-42b9-a379-2b7b440184c1", - "resource": { - "resourceType": "Observation", - "id": "9abdda58-25c5-42b9-a379-2b7b440184c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.71, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0f7854fc-dec6-4d76-8c66-6e2f512da711", - "resource": { - "resourceType": "Observation", - "id": "0f7854fc-dec6-4d76-8c66-6e2f512da711", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 8.59, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3e46da58-5a10-4449-8a72-039c0e325b27", - "resource": { - "resourceType": "Observation", - "id": "3e46da58-5a10-4449-8a72-039c0e325b27", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 141.62, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35cef2f5-8c00-4c9f-a524-0c653f2a9a73", - "resource": { - "resourceType": "Observation", - "id": "35cef2f5-8c00-4c9f-a524-0c653f2a9a73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 3.81, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:395b3672-b878-4e1e-a582-c29cf8915688", - "resource": { - "resourceType": "Observation", - "id": "395b3672-b878-4e1e-a582-c29cf8915688", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 110.09, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e636cdd-9913-4985-a409-aaa700fc07d6", - "resource": { - "resourceType": "Observation", - "id": "6e636cdd-9913-4985-a409-aaa700fc07d6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.04, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5b8e582a-ec17-493e-a837-170cdab88510", - "resource": { - "resourceType": "Observation", - "id": "5b8e582a-ec17-493e-a837-170cdab88510", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 197.31, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e73dd94a-0eb9-4077-9a21-06c0c8591b6b", - "resource": { - "resourceType": "Observation", - "id": "e73dd94a-0eb9-4077-9a21-06c0c8591b6b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 120.68, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8fb5417-cd77-44b9-94a3-72dbe33268a9", - "resource": { - "resourceType": "Observation", - "id": "d8fb5417-cd77-44b9-94a3-72dbe33268a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 105.69, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9309f308-a016-44bd-993d-53b024c8e6d6", - "resource": { - "resourceType": "Observation", - "id": "9309f308-a016-44bd-993d-53b024c8e6d6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 67.48, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5997de0b-7dac-4b65-973b-13e48a70e242", - "resource": { - "resourceType": "Observation", - "id": "5997de0b-7dac-4b65-973b-13e48a70e242", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.6333, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", - "resource": { - "resourceType": "Observation", - "id": "8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.8747, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65f5c466-b967-4a25-8b30-46aa7f632247", - "resource": { - "resourceType": "Observation", - "id": "65f5c466-b967-4a25-8b30-46aa7f632247", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 17.009, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:214ce259-d519-4273-9e68-7cf6216cbd7c", - "resource": { - "resourceType": "Observation", - "id": "214ce259-d519-4273-9e68-7cf6216cbd7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 43.181, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fffe3446-76bd-451b-8613-44811836bef8", - "resource": { - "resourceType": "Observation", - "id": "fffe3446-76bd-451b-8613-44811836bef8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 90.459, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1701017-6395-429d-afa6-baa7442b690f", - "resource": { - "resourceType": "Observation", - "id": "a1701017-6395-429d-afa6-baa7442b690f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.263, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e4335d63-a06f-4955-b199-74e791efbf54", - "resource": { - "resourceType": "Observation", - "id": "e4335d63-a06f-4955-b199-74e791efbf54", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 33.812, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", - "resource": { - "resourceType": "Observation", - "id": "a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 43.011, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83ecfdd5-9879-41c1-80bb-f929cabf55e1", - "resource": { - "resourceType": "Observation", - "id": "83ecfdd5-9879-41c1-80bb-f929cabf55e1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 210.45, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:022dd7d5-86e4-4736-a64d-d46b5a142983", - "resource": { - "resourceType": "Observation", - "id": "022dd7d5-86e4-4736-a64d-d46b5a142983", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 238.39, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e993cda-6614-4dda-b26b-1b39e3e66feb", - "resource": { - "resourceType": "Observation", - "id": "6e993cda-6614-4dda-b26b-1b39e3e66feb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 11.506, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c5538e6d-377f-4a1a-bf79-d7087505d02f", - "resource": { - "resourceType": "Observation", - "id": "c5538e6d-377f-4a1a-bf79-d7087505d02f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:944eb7a4-9849-4bd8-9897-63af2f11366c", - "resource": { - "resourceType": "Observation", - "id": "944eb7a4-9849-4bd8-9897-63af2f11366c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.14, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2726655b-6af7-4bb7-a5ca-c0d51b9b86ef", - "resource": { - "resourceType": "Procedure", - "id": "2726655b-6af7-4bb7-a5ca-c0d51b9b86ef", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "performedPeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:30:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:640affe1-0ab4-4e50-bca0-1d1d30b3433d", - "resource": { - "resourceType": "MedicationRequest", - "id": "640affe1-0ab4-4e50-bca0-1d1d30b3433d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "authoredOn": "2013-11-04T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:adc24942-4e53-4fd3-877e-c5d06ca25bc7", - "resource": { - "resourceType": "Claim", - "id": "adc24942-4e53-4fd3-877e-c5d06ca25bc7", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "created": "2013-11-04T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:640affe1-0ab4-4e50-bca0-1d1d30b3433d" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:66ba38cf-4f6f-4a75-8859-e646409be000", - "resource": { - "resourceType": "Immunization", - "id": "66ba38cf-4f6f-4a75-8859-e646409be000", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "occurrenceDateTime": "2013-11-04T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:749ca395-43c7-4f67-b6f4-9b805343b108", - "resource": { - "resourceType": "Immunization", - "id": "749ca395-43c7-4f67-b6f4-9b805343b108", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "33", - "display": "pneumococcal polysaccharide vaccine, 23 valent" - } - ], - "text": "pneumococcal polysaccharide vaccine, 23 valent" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "occurrenceDateTime": "2013-11-04T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:61e8b09a-d2e5-4a5e-a979-8ce1e1d2a86c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "61e8b09a-d2e5-4a5e-a979-8ce1e1d2a86c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:bd4b198f-5e03-44ce-978c-ee474501fc5f", - "display": "Glucose" - }, - { - "reference": "urn:uuid:01c7a467-8130-4c85-9a6a-c5cd25140ccb", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:9abdda58-25c5-42b9-a379-2b7b440184c1", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:0f7854fc-dec6-4d76-8c66-6e2f512da711", - "display": "Calcium" - }, - { - "reference": "urn:uuid:3e46da58-5a10-4449-8a72-039c0e325b27", - "display": "Sodium" - }, - { - "reference": "urn:uuid:35cef2f5-8c00-4c9f-a524-0c653f2a9a73", - "display": "Potassium" - }, - { - "reference": "urn:uuid:395b3672-b878-4e1e-a582-c29cf8915688", - "display": "Chloride" - }, - { - "reference": "urn:uuid:6e636cdd-9913-4985-a409-aaa700fc07d6", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:6d148349-8907-48f4-9011-f02be63deea9", - "resource": { - "resourceType": "DiagnosticReport", - "id": "6d148349-8907-48f4-9011-f02be63deea9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:5b8e582a-ec17-493e-a837-170cdab88510", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:e73dd94a-0eb9-4077-9a21-06c0c8591b6b", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:d8fb5417-cd77-44b9-94a3-72dbe33268a9", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:9309f308-a016-44bd-993d-53b024c8e6d6", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:6e2d9695-c690-44b6-80f8-a552dbee6d07", - "resource": { - "resourceType": "DiagnosticReport", - "id": "6e2d9695-c690-44b6-80f8-a552dbee6d07", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - }, - "effectiveDateTime": "2013-11-04T04:15:51-08:00", - "issued": "2013-11-04T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:5997de0b-7dac-4b65-973b-13e48a70e242", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:8d9ffc30-0ed4-4f7e-86b7-3a62873d7685", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:65f5c466-b967-4a25-8b30-46aa7f632247", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:214ce259-d519-4273-9e68-7cf6216cbd7c", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:fffe3446-76bd-451b-8613-44811836bef8", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:a1701017-6395-429d-afa6-baa7442b690f", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:e4335d63-a06f-4955-b199-74e791efbf54", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:a93cd321-c21e-48e9-a8f8-f6f664e5e7b8", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:83ecfdd5-9879-41c1-80bb-f929cabf55e1", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:022dd7d5-86e4-4736-a64d-d46b5a142983", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:6e993cda-6614-4dda-b26b-1b39e3e66feb", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:95f72712-d5b1-4050-af79-829974f2b7cc", - "resource": { - "resourceType": "Claim", - "id": "95f72712-d5b1-4050-af79-829974f2b7cc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "created": "2013-11-04T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:66ba38cf-4f6f-4a75-8859-e646409be000" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:749ca395-43c7-4f67-b6f4-9b805343b108" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:2726655b-6af7-4bb7-a5ca-c0d51b9b86ef" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "33", - "display": "pneumococcal polysaccharide vaccine, 23 valent" - } - ], - "text": "pneumococcal polysaccharide vaccine, 23 valent" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 456.60, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d5b1fb1d-a0cf-4659-834a-cb9066c375c0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d5b1fb1d-a0cf-4659-834a-cb9066c375c0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "95f72712-d5b1-4050-af79-829974f2b7cc" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-11-04T04:45:51-08:00", - "end": "2014-11-04T04:45:51-08:00" - }, - "created": "2013-11-04T04:45:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:95f72712-d5b1-4050-af79-829974f2b7cc" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:5bb6bc1e-129b-4f1f-bfec-f0c12d9c1455" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "33", - "display": "pneumococcal polysaccharide vaccine, 23 valent" - } - ], - "text": "pneumococcal polysaccharide vaccine, 23 valent" - }, - "servicedPeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-04T04:15:51-08:00", - "end": "2013-11-04T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 456.60, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 91.32000000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 365.28000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 456.60, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 456.60, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 590.1120000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441", - "resource": { - "resourceType": "Encounter", - "id": "b14cf9f8-7078-4d99-8753-417c656a7441", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "86013001", - "display": "Periodic reevaluation and management of healthy individual (procedure)" - } - ], - "text": "Periodic reevaluation and management of healthy individual (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "resource": { - "resourceType": "Condition", - "id": "89830890-3c8f-4911-8f8d-454837614455", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ], - "text": "Malignant neoplasm of breast (disorder)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "onsetDateTime": "2013-11-22T04:15:51-08:00", - "recordedDate": "2013-11-22T04:15:51-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:0c61e1c3-148a-4d6f-aec7-cb83bebe4300", - "resource": { - "resourceType": "Observation", - "id": "0c61e1c3-148a-4d6f-aec7-cb83bebe4300", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "44667-4", - "display": "Site of distant metastasis in Breast tumor" - } - ], - "text": "Site of distant metastasis in Breast tumor" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "260413007", - "display": "None (qualifier value)" - } - ], - "text": "None (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:608c342f-3431-438e-b8a2-4bed67b9e504", - "resource": { - "resourceType": "Observation", - "id": "608c342f-3431-438e-b8a2-4bed67b9e504", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21907-1", - "display": "Distant metastases.clinical [Class] Cancer" - } - ], - "text": "Distant metastases.clinical [Class] Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "30893008", - "display": "M0 category (finding)" - } - ], - "text": "M0 category (finding)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c657df3f-48ef-4f6b-b190-5fb6707f7e85", - "resource": { - "resourceType": "Observation", - "id": "c657df3f-48ef-4f6b-b190-5fb6707f7e85", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "33728-7", - "display": "Size.maximum dimension in Tumor" - } - ], - "text": "Size.maximum dimension in Tumor" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.022949, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1824225b-c940-476e-a2da-030dc2eeeaad", - "resource": { - "resourceType": "Observation", - "id": "1824225b-c940-476e-a2da-030dc2eeeaad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21905-5", - "display": "Primary tumor.clinical [Class] Cancer" - } - ], - "text": "Primary tumor.clinical [Class] Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23351008", - "display": "T1 category (finding)" - } - ], - "text": "T1 category (finding)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b59905f1-ad49-46cd-9257-d3b907d14a12", - "resource": { - "resourceType": "Observation", - "id": "b59905f1-ad49-46cd-9257-d3b907d14a12", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85352-3", - "display": "Lymph nodes with isolated tumor cells [#] in Cancer specimen by Light microscopy" - } - ], - "text": "Lymph nodes with isolated tumor cells [#] in Cancer specimen by Light microscopy" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueQuantity": { - "value": 0, - "unit": "#", - "system": "http://unitsofmeasure.org", - "code": "#" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b40588a2-94a4-4857-b305-153c711715e6", - "resource": { - "resourceType": "Observation", - "id": "b40588a2-94a4-4857-b305-153c711715e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "imaging", - "display": "imaging" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21906-3", - "display": "Regional lymph nodes.clinical [Class] Cancer" - } - ], - "text": "Regional lymph nodes.clinical [Class] Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "62455006", - "display": "N0 category (finding)" - } - ], - "text": "N0 category (finding)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2d03aae-348e-4615-b33e-f460985ad708", - "resource": { - "resourceType": "Observation", - "id": "a2d03aae-348e-4615-b33e-f460985ad708", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85319-2", - "display": "HER2 [Presence] in Breast cancer specimen by Immune stain" - } - ], - "text": "HER2 [Presence] in Breast cancer specimen by Immune stain" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "260385009", - "display": "Negative (qualifier value)" - } - ], - "text": "Negative (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d69bc4f6-2881-449d-9f25-9dfa218f559c", - "resource": { - "resourceType": "Observation", - "id": "d69bc4f6-2881-449d-9f25-9dfa218f559c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85318-4", - "display": "HER2 [Presence] in Breast cancer specimen by FISH" - } - ], - "text": "HER2 [Presence] in Breast cancer specimen by FISH" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueString": "less than 1.8" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:234a71bd-87ca-431e-9e07-f7d9433a9c60", - "resource": { - "resourceType": "Observation", - "id": "234a71bd-87ca-431e-9e07-f7d9433a9c60", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85337-4", - "display": "Estrogen receptor Ag [Presence] in Breast cancer specimen by Immune stain" - } - ], - "text": "Estrogen receptor Ag [Presence] in Breast cancer specimen by Immune stain" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "260385009", - "display": "Negative (qualifier value)" - } - ], - "text": "Negative (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:17d32508-2449-44dc-8829-1e371bfb905e", - "resource": { - "resourceType": "Observation", - "id": "17d32508-2449-44dc-8829-1e371bfb905e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85339-0", - "display": "Progesterone receptor Ag [Presence] in Breast cancer specimen by Immune stain" - } - ], - "text": "Progesterone receptor Ag [Presence] in Breast cancer specimen by Immune stain" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10828004", - "display": "Positive (qualifier value)" - } - ], - "text": "Positive (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e07d6bd-9d81-463d-968f-6ab4243d11b9", - "resource": { - "resourceType": "Observation", - "id": "6e07d6bd-9d81-463d-968f-6ab4243d11b9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21908-9", - "display": "Stage group.clinical Cancer" - } - ], - "text": "Stage group.clinical Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "261634002", - "display": "Stage 1A (qualifier value)" - } - ], - "text": "Stage 1A (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f2a9021-0944-49de-810c-0207545d5cc4", - "resource": { - "resourceType": "Observation", - "id": "1f2a9021-0944-49de-810c-0207545d5cc4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21908-9", - "display": "Stage group.clinical Cancer" - } - ], - "text": "Stage group.clinical Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "effectiveDateTime": "2013-11-22T04:15:51-08:00", - "issued": "2013-11-22T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "258215001", - "display": "Stage 1 (qualifier value)" - } - ], - "text": "Stage 1 (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d9bcd92-efb2-4ec2-8e94-2960f2118652", - "resource": { - "resourceType": "Procedure", - "id": "2d9bcd92-efb2-4ec2-8e94-2960f2118652", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "24623002", - "display": "Screening mammography (procedure)" - } - ], - "text": "Screening mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "performedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-22T04:35:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7bdf4525-68b7-42a1-8f40-c5f5bbbfab70", - "resource": { - "resourceType": "Procedure", - "id": "7bdf4525-68b7-42a1-8f40-c5f5bbbfab70", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "1571000087109", - "display": "Ultrasonography of bilateral breasts (procedure)" - } - ], - "text": "Ultrasonography of bilateral breasts (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "performedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-22T04:38:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:6433b673-1bf2-4b90-aeb2-ada4c8acd7fe", - "resource": { - "resourceType": "Procedure", - "id": "6433b673-1bf2-4b90-aeb2-ada4c8acd7fe", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "122548005", - "display": "Biopsy of breast (procedure)" - } - ], - "text": "Biopsy of breast (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "performedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-22T05:12:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7b29f70c-405e-4624-b4a3-fc3d8fdb243c", - "resource": { - "resourceType": "Procedure", - "id": "7b29f70c-405e-4624-b4a3-fc3d8fdb243c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "434363004", - "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "performedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a097d93f-d092-4a6e-957d-7ea09fb4c5c6", - "resource": { - "resourceType": "Procedure", - "id": "a097d93f-d092-4a6e-957d-7ea09fb4c5c6", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "433114000", - "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - }, - "performedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-22T06:15:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:404399f8-1e31-48a9-acf4-75803f40e1ac", - "resource": { - "resourceType": "Claim", - "id": "404399f8-1e31-48a9-acf4-75803f40e1ac", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "created": "2013-11-23T04:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:2d9bcd92-efb2-4ec2-8e94-2960f2118652" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:7bdf4525-68b7-42a1-8f40-c5f5bbbfab70" - } - }, - { - "sequence": 3, - "procedureReference": { - "reference": "urn:uuid:6433b673-1bf2-4b90-aeb2-ada4c8acd7fe" - } - }, - { - "sequence": 4, - "procedureReference": { - "reference": "urn:uuid:7b29f70c-405e-4624-b4a3-fc3d8fdb243c" - } - }, - { - "sequence": 5, - "procedureReference": { - "reference": "urn:uuid:a097d93f-d092-4a6e-957d-7ea09fb4c5c6" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "86013001", - "display": "Periodic reevaluation and management of healthy individual (procedure)" - } - ], - "text": "Periodic reevaluation and management of healthy individual (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "24623002", - "display": "Screening mammography (procedure)" - } - ], - "text": "Screening mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "1571000087109", - "display": "Ultrasonography of bilateral breasts (procedure)" - } - ], - "text": "Ultrasonography of bilateral breasts (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "122548005", - "display": "Biopsy of breast (procedure)" - } - ], - "text": "Biopsy of breast (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - }, - { - "sequence": 5, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ], - "text": "Malignant neoplasm of breast (disorder)" - } - }, - { - "sequence": 6, - "procedureSequence": [ - 4 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "434363004", - "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - }, - { - "sequence": 7, - "procedureSequence": [ - 5 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "433114000", - "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8a5c7307-4cab-44e9-aff8-2650b226dd50", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8a5c7307-4cab-44e9-aff8-2650b226dd50", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "404399f8-1e31-48a9-acf4-75803f40e1ac" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-11-23T04:15:51-08:00", - "end": "2014-11-23T04:15:51-08:00" - }, - "created": "2013-11-23T04:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:404399f8-1e31-48a9-acf4-75803f40e1ac" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "86013001", - "display": "Periodic reevaluation and management of healthy individual (procedure)" - } - ], - "text": "Periodic reevaluation and management of healthy individual (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b14cf9f8-7078-4d99-8753-417c656a7441" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "24623002", - "display": "Screening mammography (procedure)" - } - ], - "text": "Screening mammography (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "1571000087109", - "display": "Ultrasonography of bilateral breasts (procedure)" - } - ], - "text": "Ultrasonography of bilateral breasts (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "122548005", - "display": "Biopsy of breast (procedure)" - } - ], - "text": "Biopsy of breast (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ], - "text": "Malignant neoplasm of breast (disorder)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 6, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "434363004", - "display": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by fluorescence in situ hybridization (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 7, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "433114000", - "display": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - } - ], - "text": "Human epidermal growth factor receptor 2 gene detection by immunohistochemistry (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-22T04:15:51-08:00", - "end": "2013-11-23T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 2066.6, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50", - "resource": { - "resourceType": "Encounter", - "id": "dada8fdb-4fee-4e34-8851-6bede0d3ce50", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223484005", - "display": "Discussion about treatment (procedure)" - } - ], - "text": "Discussion about treatment (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-11-24T04:15:51-08:00", - "end": "2013-11-24T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-11-24T04:15:51-08:00", - "end": "2013-11-24T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7a4b7a71-215d-4d12-9471-cac178e23dcb", - "resource": { - "resourceType": "Claim", - "id": "7a4b7a71-215d-4d12-9471-cac178e23dcb", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-11-24T04:15:51-08:00", - "end": "2013-11-24T04:30:51-08:00" - }, - "created": "2013-11-24T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223484005", - "display": "Discussion about treatment (procedure)" - } - ], - "text": "Discussion about treatment (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0f9d824e-aed8-49e9-9fa9-bc10333bcd73", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0f9d824e-aed8-49e9-9fa9-bc10333bcd73", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7a4b7a71-215d-4d12-9471-cac178e23dcb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-11-24T04:30:51-08:00", - "end": "2014-11-24T04:30:51-08:00" - }, - "created": "2013-11-24T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7a4b7a71-215d-4d12-9471-cac178e23dcb" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223484005", - "display": "Discussion about treatment (procedure)" - } - ], - "text": "Discussion about treatment (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-24T04:15:51-08:00", - "end": "2013-11-24T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:dada8fdb-4fee-4e34-8851-6bede0d3ce50" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5", - "resource": { - "resourceType": "Encounter", - "id": "b542b686-f1d2-4373-a1bf-8e1b4b36d7a5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-27T06:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-27T06:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:553a9230-0e12-44c4-96be-8a1357afe96d", - "resource": { - "resourceType": "Procedure", - "id": "553a9230-0e12-44c4-96be-8a1357afe96d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "69031006", - "display": "Excision of breast tissue (procedure)" - } - ], - "text": "Excision of breast tissue (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" - }, - "performedPeriod": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-26T06:15:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:d05bf801-8a4f-487a-9a28-2672ee2973a1", - "resource": { - "resourceType": "Claim", - "id": "d05bf801-8a4f-487a-9a28-2672ee2973a1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-27T06:15:51-08:00" - }, - "created": "2013-11-27T06:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:553a9230-0e12-44c4-96be-8a1357afe96d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "69031006", - "display": "Excision of breast tissue (procedure)" - } - ], - "text": "Excision of breast tissue (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bd123d23-56e2-4825-ab16-6734a0d642cd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "bd123d23-56e2-4825-ab16-6734a0d642cd", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d05bf801-8a4f-487a-9a28-2672ee2973a1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-11-27T06:15:51-08:00", - "end": "2014-11-27T06:15:51-08:00" - }, - "created": "2013-11-27T06:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:d05bf801-8a4f-487a-9a28-2672ee2973a1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-27T06:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b542b686-f1d2-4373-a1bf-8e1b4b36d7a5" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "69031006", - "display": "Excision of breast tissue (procedure)" - } - ], - "text": "Excision of breast tissue (procedure)" - }, - "servicedPeriod": { - "start": "2013-11-26T04:15:51-08:00", - "end": "2013-11-27T06:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71", - "resource": { - "resourceType": "Encounter", - "id": "228f3cbf-e4fe-4f71-a4e8-0c494fd44d71", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:331cf1fe-c9ea-4cf7-98b3-2ac10deea32c", - "resource": { - "resourceType": "Procedure", - "id": "331cf1fe-c9ea-4cf7-98b3-2ac10deea32c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" - }, - "performedPeriod": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-05T04:15:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b8e9bf83-736c-4a49-802b-f2bf65dc5fdb", - "resource": { - "resourceType": "MedicationRequest", - "id": "b8e9bf83-736c-4a49-802b-f2bf65dc5fdb", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" - }, - "authoredOn": "2013-12-05T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:27c611a5-0a2c-4ff4-8fdc-c014dce6e23e", - "resource": { - "resourceType": "Claim", - "id": "27c611a5-0a2c-4ff4-8fdc-c014dce6e23e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "created": "2013-12-06T04:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:b8e9bf83-736c-4a49-802b-f2bf65dc5fdb" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" - } - ] - } - ], - "total": { - "value": 6470.39, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:040ddc05-c3c4-4a78-9e59-7dd25634607e", - "resource": { - "resourceType": "Claim", - "id": "040ddc05-c3c4-4a78-9e59-7dd25634607e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "created": "2013-12-06T04:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:331cf1fe-c9ea-4cf7-98b3-2ac10deea32c" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2650dbb5-1e6b-43d1-bba2-5f03860a177f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2650dbb5-1e6b-43d1-bba2-5f03860a177f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "040ddc05-c3c4-4a78-9e59-7dd25634607e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-12-06T04:15:51-08:00", - "end": "2014-12-06T04:15:51-08:00" - }, - "created": "2013-12-06T04:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:040ddc05-c3c4-4a78-9e59-7dd25634607e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:228f3cbf-e4fe-4f71-a4e8-0c494fd44d71" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-12-05T04:15:51-08:00", - "end": "2013-12-06T04:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb", - "resource": { - "resourceType": "Encounter", - "id": "16c7951f-92b6-44ac-b75a-1323d58319fb", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c332b370-eb9d-4ee3-9a6f-060c6e888330", - "resource": { - "resourceType": "Procedure", - "id": "c332b370-eb9d-4ee3-9a6f-060c6e888330", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" - }, - "performedPeriod": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-27T05:15:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:55c2af1b-0077-4dcd-ac3c-a4446e76ff88", - "resource": { - "resourceType": "MedicationRequest", - "id": "55c2af1b-0077-4dcd-ac3c-a4446e76ff88", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" - }, - "authoredOn": "2013-12-27T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:b433dbc1-938d-46d2-9934-75fa59e23583", - "resource": { - "resourceType": "Claim", - "id": "b433dbc1-938d-46d2-9934-75fa59e23583", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "created": "2013-12-28T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:55c2af1b-0077-4dcd-ac3c-a4446e76ff88" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" - } - ] - } - ], - "total": { - "value": 6389.43, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fd2c5e5b-d3a1-47c9-84e9-0d957881497f", - "resource": { - "resourceType": "Claim", - "id": "fd2c5e5b-d3a1-47c9-84e9-0d957881497f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "created": "2013-12-28T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:c332b370-eb9d-4ee3-9a6f-060c6e888330" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:65dc99d0-29bc-4cbb-b24b-b0da621c633a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "65dc99d0-29bc-4cbb-b24b-b0da621c633a", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fd2c5e5b-d3a1-47c9-84e9-0d957881497f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2013-12-28T05:15:51-08:00", - "end": "2014-12-28T05:15:51-08:00" - }, - "created": "2013-12-28T05:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:fd2c5e5b-d3a1-47c9-84e9-0d957881497f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:16c7951f-92b6-44ac-b75a-1323d58319fb" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2013-12-27T04:15:51-08:00", - "end": "2013-12-28T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c", - "resource": { - "resourceType": "Encounter", - "id": "8c4f2b96-5ca1-4014-8945-b77541808e6c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d7f60c87-8fff-4f41-b97b-bb5477f09efe", - "resource": { - "resourceType": "Procedure", - "id": "d7f60c87-8fff-4f41-b97b-bb5477f09efe", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" - }, - "performedPeriod": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-16T05:15:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:fcd92e33-245f-48d9-a7a3-d8b61f4d2a38", - "resource": { - "resourceType": "MedicationRequest", - "id": "fcd92e33-245f-48d9-a7a3-d8b61f4d2a38", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" - }, - "authoredOn": "2014-01-16T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:3767dc7d-1c1f-4cf8-982e-59c4c9d79113", - "resource": { - "resourceType": "Claim", - "id": "3767dc7d-1c1f-4cf8-982e-59c4c9d79113", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "created": "2014-01-17T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:fcd92e33-245f-48d9-a7a3-d8b61f4d2a38" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" - } - ] - } - ], - "total": { - "value": 6515.37, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:96a9eeb6-9b95-4f26-9166-851a5262fcd1", - "resource": { - "resourceType": "Claim", - "id": "96a9eeb6-9b95-4f26-9166-851a5262fcd1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "created": "2014-01-17T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:d7f60c87-8fff-4f41-b97b-bb5477f09efe" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d87eac2a-83ba-451f-86fb-b522336977cb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d87eac2a-83ba-451f-86fb-b522336977cb", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "96a9eeb6-9b95-4f26-9166-851a5262fcd1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-01-17T05:15:51-08:00", - "end": "2015-01-17T05:15:51-08:00" - }, - "created": "2014-01-17T05:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:96a9eeb6-9b95-4f26-9166-851a5262fcd1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:8c4f2b96-5ca1-4014-8945-b77541808e6c" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-01-16T04:15:51-08:00", - "end": "2014-01-17T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a", - "resource": { - "resourceType": "Encounter", - "id": "b7493584-25de-4839-894c-dec5e46a500a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5673bd55-4554-4174-ad19-191760da1d52", - "resource": { - "resourceType": "Procedure", - "id": "5673bd55-4554-4174-ad19-191760da1d52", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" - }, - "performedPeriod": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-07T05:15:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:2d84e83e-18a3-4636-9c3c-0442cbf4c037", - "resource": { - "resourceType": "MedicationRequest", - "id": "2d84e83e-18a3-4636-9c3c-0442cbf4c037", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" - }, - "authoredOn": "2014-02-07T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:550a3cc6-ff13-481b-a696-10ed812d9de4", - "resource": { - "resourceType": "Claim", - "id": "550a3cc6-ff13-481b-a696-10ed812d9de4", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "created": "2014-02-08T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:2d84e83e-18a3-4636-9c3c-0442cbf4c037" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" - } - ] - } - ], - "total": { - "value": 6289.71, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:63563296-992c-40b8-bee8-fc4908e779d5", - "resource": { - "resourceType": "Claim", - "id": "63563296-992c-40b8-bee8-fc4908e779d5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "created": "2014-02-08T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:5673bd55-4554-4174-ad19-191760da1d52" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5903532d-1c7d-4c74-b547-2fdd0911e42d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5903532d-1c7d-4c74-b547-2fdd0911e42d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "63563296-992c-40b8-bee8-fc4908e779d5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-02-08T05:15:51-08:00", - "end": "2015-02-08T05:15:51-08:00" - }, - "created": "2014-02-08T05:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:63563296-992c-40b8-bee8-fc4908e779d5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b7493584-25de-4839-894c-dec5e46a500a" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-02-07T04:15:51-08:00", - "end": "2014-02-08T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8", - "resource": { - "resourceType": "Encounter", - "id": "f05e59fc-b1f2-4ef4-b29f-30c0860d86b8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:eac54ae8-28e3-4372-9343-408b7e9eb936", - "resource": { - "resourceType": "Procedure", - "id": "eac54ae8-28e3-4372-9343-408b7e9eb936", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" - }, - "performedPeriod": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-02-28T05:15:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:6a3de65f-6f05-4166-8b8a-34657d72794d", - "resource": { - "resourceType": "MedicationRequest", - "id": "6a3de65f-6f05-4166-8b8a-34657d72794d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" - }, - "authoredOn": "2014-02-28T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:950faa81-e44f-481e-878a-be9454913c9f", - "resource": { - "resourceType": "Claim", - "id": "950faa81-e44f-481e-878a-be9454913c9f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "created": "2014-03-01T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:6a3de65f-6f05-4166-8b8a-34657d72794d" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" - } - ] - } - ], - "total": { - "value": 6623.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2633abc9-222b-4a73-9f00-599bf2a5d7bd", - "resource": { - "resourceType": "Claim", - "id": "2633abc9-222b-4a73-9f00-599bf2a5d7bd", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "created": "2014-03-01T05:15:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:eac54ae8-28e3-4372-9343-408b7e9eb936" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1bccd126-9798-41da-9f59-0f0e58db4ef8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1bccd126-9798-41da-9f59-0f0e58db4ef8", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2633abc9-222b-4a73-9f00-599bf2a5d7bd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-03-01T05:15:51-08:00", - "end": "2015-03-01T05:15:51-08:00" - }, - "created": "2014-03-01T05:15:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2633abc9-222b-4a73-9f00-599bf2a5d7bd" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f05e59fc-b1f2-4ef4-b29f-30c0860d86b8" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-02-28T04:15:51-08:00", - "end": "2014-03-01T05:15:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d", - "resource": { - "resourceType": "Encounter", - "id": "51e879d8-de42-4413-88c1-4e24fae5803d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e69dbe01-13f8-4dd9-8f1b-54488e359385", - "resource": { - "resourceType": "Procedure", - "id": "e69dbe01-13f8-4dd9-8f1b-54488e359385", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" - }, - "performedPeriod": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-21T06:15:51-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:11df9d03-997a-4999-ac9b-915c7533be1f", - "resource": { - "resourceType": "MedicationRequest", - "id": "11df9d03-997a-4999-ac9b-915c7533be1f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" - }, - "authoredOn": "2014-03-21T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5e345c16-e322-47bc-9b7a-a5c746a02d23", - "resource": { - "resourceType": "Claim", - "id": "5e345c16-e322-47bc-9b7a-a5c746a02d23", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "created": "2014-03-22T06:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:11df9d03-997a-4999-ac9b-915c7533be1f" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" - } - ] - } - ], - "total": { - "value": 6500.38, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:64da860e-8923-438a-bdc5-d18c4fc7c17a", - "resource": { - "resourceType": "Claim", - "id": "64da860e-8923-438a-bdc5-d18c4fc7c17a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "created": "2014-03-22T06:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e69dbe01-13f8-4dd9-8f1b-54488e359385" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e8d72d04-3a14-4dd0-96c4-34d6f5c5c05d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e8d72d04-3a14-4dd0-96c4-34d6f5c5c05d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "64da860e-8923-438a-bdc5-d18c4fc7c17a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-03-22T06:15:51-07:00", - "end": "2015-03-22T06:15:51-07:00" - }, - "created": "2014-03-22T06:15:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:64da860e-8923-438a-bdc5-d18c4fc7c17a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:51e879d8-de42-4413-88c1-4e24fae5803d" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-03-21T05:15:51-07:00", - "end": "2014-03-22T06:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5", - "resource": { - "resourceType": "Encounter", - "id": "328a23f5-ab37-4cd5-aa29-092180c8bfc5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3027d537-9a35-4aee-8f0f-0d08352a222d", - "resource": { - "resourceType": "Procedure", - "id": "3027d537-9a35-4aee-8f0f-0d08352a222d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" - }, - "performedPeriod": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-10T06:15:51-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:014e5434-d1c6-4251-a6f4-06635b85f3d2", - "resource": { - "resourceType": "MedicationRequest", - "id": "014e5434-d1c6-4251-a6f4-06635b85f3d2", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" - }, - "authoredOn": "2014-04-10T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:2a3893a6-bdbb-476b-8b5b-1acde08b71f7", - "resource": { - "resourceType": "Claim", - "id": "2a3893a6-bdbb-476b-8b5b-1acde08b71f7", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "created": "2014-04-11T06:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:014e5434-d1c6-4251-a6f4-06635b85f3d2" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" - } - ] - } - ], - "total": { - "value": 6432.3, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2a7128b6-73ed-443f-82be-2ada7529b87a", - "resource": { - "resourceType": "Claim", - "id": "2a7128b6-73ed-443f-82be-2ada7529b87a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "created": "2014-04-11T06:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:3027d537-9a35-4aee-8f0f-0d08352a222d" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:10f99655-874d-4dae-8f54-0187a578cba5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "10f99655-874d-4dae-8f54-0187a578cba5", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2a7128b6-73ed-443f-82be-2ada7529b87a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-04-11T06:15:51-07:00", - "end": "2015-04-11T06:15:51-07:00" - }, - "created": "2014-04-11T06:15:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:2a7128b6-73ed-443f-82be-2ada7529b87a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:328a23f5-ab37-4cd5-aa29-092180c8bfc5" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-04-10T05:15:51-07:00", - "end": "2014-04-11T06:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54", - "resource": { - "resourceType": "Encounter", - "id": "e6d36055-a5bc-434f-ba14-70b08f1ddc54", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6fbd1be0-04c7-420b-ba56-a213fab757ad", - "resource": { - "resourceType": "Procedure", - "id": "6fbd1be0-04c7-420b-ba56-a213fab757ad", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" - }, - "performedPeriod": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-01T05:15:51-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e9fa9475-38ce-4e90-a55d-afc3c0f1bfab", - "resource": { - "resourceType": "MedicationRequest", - "id": "e9fa9475-38ce-4e90-a55d-afc3c0f1bfab", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "583214", - "display": "Paclitaxel 100 MG Injection" - } - ], - "text": "Paclitaxel 100 MG Injection" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" - }, - "authoredOn": "2014-05-01T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:781787e7-9169-4fc9-b955-a2d45b874b08", - "resource": { - "resourceType": "Claim", - "id": "781787e7-9169-4fc9-b955-a2d45b874b08", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "created": "2014-05-02T05:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:e9fa9475-38ce-4e90-a55d-afc3c0f1bfab" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" - } - ] - } - ], - "total": { - "value": 6906.05, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3764e02f-7125-4610-a1fe-3d255b6df4b9", - "resource": { - "resourceType": "Claim", - "id": "3764e02f-7125-4610-a1fe-3d255b6df4b9", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "created": "2014-05-02T05:15:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:6fbd1be0-04c7-420b-ba56-a213fab757ad" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:95e4e1dd-91d1-405d-afa4-1c4cadbda3b0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "95e4e1dd-91d1-405d-afa4-1c4cadbda3b0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3764e02f-7125-4610-a1fe-3d255b6df4b9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-05-02T05:15:51-07:00", - "end": "2015-05-02T05:15:51-07:00" - }, - "created": "2014-05-02T05:15:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3764e02f-7125-4610-a1fe-3d255b6df4b9" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e6d36055-a5bc-434f-ba14-70b08f1ddc54" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "367336001", - "display": "Chemotherapy (procedure)" - } - ], - "text": "Chemotherapy (procedure)" - }, - "servicedPeriod": { - "start": "2014-05-01T05:15:51-07:00", - "end": "2014-05-02T05:15:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0", - "resource": { - "resourceType": "Encounter", - "id": "e0f6735b-abb1-4c75-8bd9-64ff8a5851a0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-05-12T05:15:51-07:00", - "end": "2014-05-12T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-05-12T05:15:51-07:00", - "end": "2014-05-12T05:30:51-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:367e3a46-b1ea-4e7e-90bc-bd74398d836e", - "resource": { - "resourceType": "Observation", - "id": "367e3a46-b1ea-4e7e-90bc-bd74398d836e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "procedure", - "display": "procedure" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59557-9", - "display": "Treatment status Cancer" - } - ], - "text": "Treatment status Cancer" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - }, - "effectiveDateTime": "2014-05-12T05:15:51-07:00", - "issued": "2014-05-12T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "445528004", - "display": "Treatment changed (situation)" - } - ], - "text": "Treatment changed (situation)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b30ae9d-2aa8-4be7-99b9-2bad98a48e5d", - "resource": { - "resourceType": "Observation", - "id": "1b30ae9d-2aa8-4be7-99b9-2bad98a48e5d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "therapy", - "display": "therapy" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "88040-1", - "display": "Response to cancer treatment" - } - ], - "text": "Response to cancer treatment" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - }, - "effectiveDateTime": "2014-05-12T05:15:51-07:00", - "issued": "2014-05-12T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385633008", - "display": "Improving (qualifier value)" - } - ], - "text": "Improving (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c55d5924-a63c-431f-8f06-107404ce70cb", - "resource": { - "resourceType": "MedicationRequest", - "id": "c55d5924-a63c-431f-8f06-107404ce70cb", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "199224", - "display": "anastrozole 1 MG Oral Tablet" - } - ], - "text": "anastrozole 1 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - }, - "authoredOn": "2014-05-12T05:15:51-07:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:02d62831-7398-4845-aa65-880e0ca6eb20", - "resource": { - "resourceType": "Claim", - "id": "02d62831-7398-4845-aa65-880e0ca6eb20", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-05-12T05:15:51-07:00", - "end": "2014-05-12T05:30:51-07:00" - }, - "created": "2014-05-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:c55d5924-a63c-431f-8f06-107404ce70cb" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c018c44a-eca8-4355-aaf2-8fe014d18b43", - "resource": { - "resourceType": "Claim", - "id": "c018c44a-eca8-4355-aaf2-8fe014d18b43", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-05-12T05:15:51-07:00", - "end": "2014-05-12T05:30:51-07:00" - }, - "created": "2014-05-12T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "encounter": [ - { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:006bd392-f7b4-432c-a080-f79e2b50780f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "006bd392-f7b4-432c-a080-f79e2b50780f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c018c44a-eca8-4355-aaf2-8fe014d18b43" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-05-12T05:30:51-07:00", - "end": "2015-05-12T05:30:51-07:00" - }, - "created": "2014-05-12T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:c018c44a-eca8-4355-aaf2-8fe014d18b43" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - }, - "servicedPeriod": { - "start": "2014-05-12T05:15:51-07:00", - "end": "2014-05-12T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e0f6735b-abb1-4c75-8bd9-64ff8a5851a0" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8", - "resource": { - "resourceType": "Encounter", - "id": "a9b8805c-69da-45a0-bfb4-e7a0a676b4a8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-08-18T05:15:51-07:00", - "end": "2014-08-18T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-08-18T05:15:51-07:00", - "end": "2014-08-18T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:07c40acd-88fd-4224-a72e-ba0263e587d5", - "resource": { - "resourceType": "Claim", - "id": "07c40acd-88fd-4224-a72e-ba0263e587d5", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-08-18T05:15:51-07:00", - "end": "2014-08-18T05:30:51-07:00" - }, - "created": "2014-08-18T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6f2abedf-483a-411d-995f-463a96654c1e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6f2abedf-483a-411d-995f-463a96654c1e", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "07c40acd-88fd-4224-a72e-ba0263e587d5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-08-18T05:30:51-07:00", - "end": "2015-08-18T05:30:51-07:00" - }, - "created": "2014-08-18T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:07c40acd-88fd-4224-a72e-ba0263e587d5" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2014-08-18T05:15:51-07:00", - "end": "2014-08-18T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a9b8805c-69da-45a0-bfb4-e7a0a676b4a8" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7", - "resource": { - "resourceType": "Encounter", - "id": "53766e80-9208-4520-9598-d6d152bd04b7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:45e8cad5-7d34-46c2-9ea9-c87f5fbe4273", - "resource": { - "resourceType": "Observation", - "id": "45e8cad5-7d34-46c2-9ea9-c87f5fbe4273", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:163e8daa-97ba-4518-bd1d-c002de45ab7c", - "resource": { - "resourceType": "Observation", - "id": "163e8daa-97ba-4518-bd1d-c002de45ab7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1b6077cf-2555-47a7-9266-db6539d86e45", - "resource": { - "resourceType": "Observation", - "id": "1b6077cf-2555-47a7-9266-db6539d86e45", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eac88c1e-ab1c-47f6-8c12-e75925934134", - "resource": { - "resourceType": "Observation", - "id": "eac88c1e-ab1c-47f6-8c12-e75925934134", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f4bc8640-a954-41a9-a33e-3840537c2978", - "resource": { - "resourceType": "Observation", - "id": "f4bc8640-a954-41a9-a33e-3840537c2978", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 125, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4dc17dee-b490-48fc-a062-72428add11a9", - "resource": { - "resourceType": "Observation", - "id": "4dc17dee-b490-48fc-a062-72428add11a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 94, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25469286-28f1-4a51-bc9a-379d021a9592", - "resource": { - "resourceType": "Observation", - "id": "25469286-28f1-4a51-bc9a-379d021a9592", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:16b81c2e-49e2-4718-a03f-b0ea902fde5a", - "resource": { - "resourceType": "Observation", - "id": "16b81c2e-49e2-4718-a03f-b0ea902fde5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 69.71, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0faca098-17c3-4cc1-9f4f-e53a482a3de7", - "resource": { - "resourceType": "Observation", - "id": "0faca098-17c3-4cc1-9f4f-e53a482a3de7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.89, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c6472ec-abc2-4c56-801c-5b007a1c0d02", - "resource": { - "resourceType": "Observation", - "id": "4c6472ec-abc2-4c56-801c-5b007a1c0d02", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.7, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d9e50bdc-a8fb-4305-9690-b361a8f310b6", - "resource": { - "resourceType": "Observation", - "id": "d9e50bdc-a8fb-4305-9690-b361a8f310b6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 8.85, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71052ac5-b9fe-45e9-95e8-e3075e573983", - "resource": { - "resourceType": "Observation", - "id": "71052ac5-b9fe-45e9-95e8-e3075e573983", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 143.17, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d9c2e28-572b-4d11-8073-178e716da083", - "resource": { - "resourceType": "Observation", - "id": "1d9c2e28-572b-4d11-8073-178e716da083", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 3.9, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", - "resource": { - "resourceType": "Observation", - "id": "02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 106.66, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69090f27-b9d4-4969-aec4-fcad583c19c5", - "resource": { - "resourceType": "Observation", - "id": "69090f27-b9d4-4969-aec4-fcad583c19c5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 26.96, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6aeb4914-9cba-4c32-ab9c-03423548dbce", - "resource": { - "resourceType": "Observation", - "id": "6aeb4914-9cba-4c32-ab9c-03423548dbce", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cb12980-f522-4eb4-b071-f6fe06afed50", - "resource": { - "resourceType": "Observation", - "id": "0cb12980-f522-4eb4-b071-f6fe06afed50", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.12, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:05574e52-2809-4e95-8df2-a700f6856f11", - "resource": { - "resourceType": "MedicationRequest", - "id": "05574e52-2809-4e95-8df2-a700f6856f11", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "authoredOn": "2014-11-10T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1d5bafe1-5c29-4036-9a5a-d8b76df7cfbf", - "resource": { - "resourceType": "Claim", - "id": "1d5bafe1-5c29-4036-9a5a-d8b76df7cfbf", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "created": "2014-11-10T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:05574e52-2809-4e95-8df2-a700f6856f11" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:06166efc-fab7-4872-8e46-b0f4bf1f5de6", - "resource": { - "resourceType": "Immunization", - "id": "06166efc-fab7-4872-8e46-b0f4bf1f5de6", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "occurrenceDateTime": "2014-11-10T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4442da8a-40bd-47b7-b640-48577dc5942c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4442da8a-40bd-47b7-b640-48577dc5942c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - }, - "effectiveDateTime": "2014-11-10T04:15:51-08:00", - "issued": "2014-11-10T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:16b81c2e-49e2-4718-a03f-b0ea902fde5a", - "display": "Glucose" - }, - { - "reference": "urn:uuid:0faca098-17c3-4cc1-9f4f-e53a482a3de7", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:4c6472ec-abc2-4c56-801c-5b007a1c0d02", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:d9e50bdc-a8fb-4305-9690-b361a8f310b6", - "display": "Calcium" - }, - { - "reference": "urn:uuid:71052ac5-b9fe-45e9-95e8-e3075e573983", - "display": "Sodium" - }, - { - "reference": "urn:uuid:1d9c2e28-572b-4d11-8073-178e716da083", - "display": "Potassium" - }, - { - "reference": "urn:uuid:02d2a632-2de5-4f64-b6d3-32ce66f3e5c9", - "display": "Chloride" - }, - { - "reference": "urn:uuid:69090f27-b9d4-4969-aec4-fcad583c19c5", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:be513f2a-5fe4-427b-8036-450e0b806f94", - "resource": { - "resourceType": "Claim", - "id": "be513f2a-5fe4-427b-8036-450e0b806f94", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "created": "2014-11-10T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:06166efc-fab7-4872-8e46-b0f4bf1f5de6" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e9cc25c0-dad7-476c-a21a-9dae3b37efa3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e9cc25c0-dad7-476c-a21a-9dae3b37efa3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "be513f2a-5fe4-427b-8036-450e0b806f94" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-11-10T04:30:51-08:00", - "end": "2015-11-10T04:30:51-08:00" - }, - "created": "2014-11-10T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:be513f2a-5fe4-427b-8036-450e0b806f94" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:53766e80-9208-4520-9598-d6d152bd04b7" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2014-11-10T04:15:51-08:00", - "end": "2014-11-10T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f", - "resource": { - "resourceType": "Encounter", - "id": "9a33c653-4d89-4e7f-b827-3ea7774e064f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2014-12-08T04:15:51-08:00", - "end": "2014-12-08T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2014-12-08T04:15:51-08:00", - "end": "2014-12-08T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9d28d844-2bc4-4680-ae36-3d1b91cfd059", - "resource": { - "resourceType": "Claim", - "id": "9d28d844-2bc4-4680-ae36-3d1b91cfd059", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2014-12-08T04:15:51-08:00", - "end": "2014-12-08T04:30:51-08:00" - }, - "created": "2014-12-08T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cd37a371-76f3-410c-8876-c5e49f432762", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cd37a371-76f3-410c-8876-c5e49f432762", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9d28d844-2bc4-4680-ae36-3d1b91cfd059" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2014-12-08T04:30:51-08:00", - "end": "2015-12-08T04:30:51-08:00" - }, - "created": "2014-12-08T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9d28d844-2bc4-4680-ae36-3d1b91cfd059" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2014-12-08T04:15:51-08:00", - "end": "2014-12-08T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:9a33c653-4d89-4e7f-b827-3ea7774e064f" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83", - "resource": { - "resourceType": "Encounter", - "id": "cc631c4c-782d-42bf-9bd4-ae8b9db2bc83", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:239f47b9-8621-415e-a4a1-d6df31f681ea", - "resource": { - "resourceType": "Claim", - "id": "239f47b9-8621-415e-a4a1-d6df31f681ea", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:30:51-07:00" - }, - "created": "2015-03-23T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0502bcb2-171b-4067-b713-cc6b181ddaf2", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0502bcb2-171b-4067-b713-cc6b181ddaf2", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "239f47b9-8621-415e-a4a1-d6df31f681ea" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-03-23T05:30:51-07:00", - "end": "2016-03-23T05:30:51-07:00" - }, - "created": "2015-03-23T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:239f47b9-8621-415e-a4a1-d6df31f681ea" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:cc631c4c-782d-42bf-9bd4-ae8b9db2bc83" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f", - "resource": { - "resourceType": "Encounter", - "id": "db4e5e28-8de2-401b-8c4b-9e678d4ed30f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-24T05:38:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-24T05:38:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945", - "resource": { - "resourceType": "Procedure", - "id": "4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" - }, - "performedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:38:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7023c212-9ed4-4522-965a-127120a89fe3", - "resource": { - "resourceType": "Claim", - "id": "7023c212-9ed4-4522-965a-127120a89fe3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-24T05:38:51-07:00" - }, - "created": "2015-03-24T05:38:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4d6b73c4-30ff-4b8e-a5cd-3ad7b3a86945" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:18ca771b-37cd-411b-8bd9-fb5e291d31cc", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "18ca771b-37cd-411b-8bd9-fb5e291d31cc", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7023c212-9ed4-4522-965a-127120a89fe3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-03-24T05:38:51-07:00", - "end": "2016-03-24T05:38:51-07:00" - }, - "created": "2015-03-24T05:38:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7023c212-9ed4-4522-965a-127120a89fe3" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-24T05:38:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:db4e5e28-8de2-401b-8c4b-9e678d4ed30f" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-24T05:38:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9", - "resource": { - "resourceType": "Encounter", - "id": "c62a7444-0148-4e6f-b341-f775b22bbfb9", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:57:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:57:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:95d309d0-1db8-4746-bc6a-01332f2d132b", - "resource": { - "resourceType": "Observation", - "id": "95d309d0-1db8-4746-bc6a-01332f2d132b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "therapy", - "display": "therapy" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "88040-1", - "display": "Response to cancer treatment" - } - ], - "text": "Response to cancer treatment" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" - }, - "effectiveDateTime": "2015-03-23T05:15:51-07:00", - "issued": "2015-03-23T05:15:51.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385633008", - "display": "Improving (qualifier value)" - } - ], - "text": "Improving (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3", - "resource": { - "resourceType": "Procedure", - "id": "ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" - }, - "performedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:42:51-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:cb1e28e0-c4a1-4843-8cc0-68caca58f6e2", - "resource": { - "resourceType": "Claim", - "id": "cb1e28e0-c4a1-4843-8cc0-68caca58f6e2", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:57:51-07:00" - }, - "created": "2015-03-23T05:57:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ba63cb0c-0d0e-4c80-8cb2-c21233e48ac3" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 15112.10, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:430b4506-96ad-4f5a-b375-d244d896ad25", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "430b4506-96ad-4f5a-b375-d244d896ad25", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cb1e28e0-c4a1-4843-8cc0-68caca58f6e2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-03-23T05:57:51-07:00", - "end": "2016-03-23T05:57:51-07:00" - }, - "created": "2015-03-23T05:57:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:cb1e28e0-c4a1-4843-8cc0-68caca58f6e2" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:57:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c62a7444-0148-4e6f-b341-f775b22bbfb9" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2015-03-23T05:15:51-07:00", - "end": "2015-03-23T05:57:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 15112.10, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 3022.42, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 12089.68, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 15112.10, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 15112.10, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 12089.68, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439", - "resource": { - "resourceType": "Encounter", - "id": "bf22e432-063f-4b85-b8fd-7774f7294439", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-07-13T05:15:51-07:00", - "end": "2015-07-13T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-07-13T05:15:51-07:00", - "end": "2015-07-13T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1", - "resource": { - "resourceType": "Claim", - "id": "aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-07-13T05:15:51-07:00", - "end": "2015-07-13T05:30:51-07:00" - }, - "created": "2015-07-13T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b9b6b7fb-6d19-4449-ae8f-43b4a6d0a2ac", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b9b6b7fb-6d19-4449-ae8f-43b4a6d0a2ac", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-07-13T05:30:51-07:00", - "end": "2016-07-13T05:30:51-07:00" - }, - "created": "2015-07-13T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:aed1d2f1-d944-4f4a-9edb-e1ffdaf0f0f1" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2015-07-13T05:15:51-07:00", - "end": "2015-07-13T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:bf22e432-063f-4b85-b8fd-7774f7294439" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b", - "resource": { - "resourceType": "Encounter", - "id": "88775039-b70f-4941-adab-6b1b01cf046b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-07-23T05:15:51-07:00", - "end": "2015-07-23T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-07-23T05:15:51-07:00", - "end": "2015-07-23T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "109838007", - "display": "Overlapping malignant neoplasm of colon" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ca8c84f5-d23b-4824-8746-f8a290a7c05e", - "resource": { - "resourceType": "Claim", - "id": "ca8c84f5-d23b-4824-8746-f8a290a7c05e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-07-23T05:15:51-07:00", - "end": "2015-07-23T05:30:51-07:00" - }, - "created": "2015-07-23T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9259701d-d0d5-405a-891a-a1199788a08e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9259701d-d0d5-405a-891a-a1199788a08e", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ca8c84f5-d23b-4824-8746-f8a290a7c05e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-07-23T05:30:51-07:00", - "end": "2016-07-23T05:30:51-07:00" - }, - "created": "2015-07-23T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ca8c84f5-d23b-4824-8746-f8a290a7c05e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2015-07-23T05:15:51-07:00", - "end": "2015-07-23T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:88775039-b70f-4941-adab-6b1b01cf046b" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1", - "resource": { - "resourceType": "Encounter", - "id": "818ecd47-9f63-4ead-8937-dae05f3079c1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-10-19T05:15:51-07:00", - "end": "2015-10-19T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2015-10-19T05:15:51-07:00", - "end": "2015-10-19T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9b72a763-0c0b-409e-9ca6-571ae5c6c15e", - "resource": { - "resourceType": "Claim", - "id": "9b72a763-0c0b-409e-9ca6-571ae5c6c15e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-10-19T05:15:51-07:00", - "end": "2015-10-19T05:30:51-07:00" - }, - "created": "2015-10-19T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4cdae89d-dc39-4a7d-9768-1918080d6f72", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4cdae89d-dc39-4a7d-9768-1918080d6f72", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9b72a763-0c0b-409e-9ca6-571ae5c6c15e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-10-19T05:30:51-07:00", - "end": "2016-10-19T05:30:51-07:00" - }, - "created": "2015-10-19T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9b72a763-0c0b-409e-9ca6-571ae5c6c15e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2015-10-19T05:15:51-07:00", - "end": "2015-10-19T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:818ecd47-9f63-4ead-8937-dae05f3079c1" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77", - "resource": { - "resourceType": "Encounter", - "id": "2583e8ac-f7b4-41b9-b9cb-e80a38d44a77", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:56a02f6a-4d0f-414c-b249-de948b9e8f20", - "resource": { - "resourceType": "Observation", - "id": "56a02f6a-4d0f-414c-b249-de948b9e8f20", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:050720d7-e226-4920-b4cf-a33561f211d4", - "resource": { - "resourceType": "Observation", - "id": "050720d7-e226-4920-b4cf-a33561f211d4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:178db7e9-e324-4932-bb75-bdc4df409b5a", - "resource": { - "resourceType": "Observation", - "id": "178db7e9-e324-4932-bb75-bdc4df409b5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d9c19adb-0eba-4bec-a156-85ce1b7ea7ec", - "resource": { - "resourceType": "Observation", - "id": "d9c19adb-0eba-4bec-a156-85ce1b7ea7ec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2e451479-0f1d-42d0-8fac-bf47e66cf564", - "resource": { - "resourceType": "Observation", - "id": "2e451479-0f1d-42d0-8fac-bf47e66cf564", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 109, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d9c6cbd1-d7dc-4419-98c1-f7d43568ad1b", - "resource": { - "resourceType": "Observation", - "id": "d9c6cbd1-d7dc-4419-98c1-f7d43568ad1b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 78, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:430ce9a9-94c8-433f-8547-837f237e5bb7", - "resource": { - "resourceType": "Observation", - "id": "430ce9a9-94c8-433f-8547-837f237e5bb7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65104e37-d772-42e9-ab0a-c793d779e36f", - "resource": { - "resourceType": "Observation", - "id": "65104e37-d772-42e9-ab0a-c793d779e36f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 74.84, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d14c63db-3889-4321-a2b9-2698d9c0e258", - "resource": { - "resourceType": "Observation", - "id": "d14c63db-3889-4321-a2b9-2698d9c0e258", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 13.23, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ae8f93b9-b1c5-4564-a8dd-882f69207ec3", - "resource": { - "resourceType": "Observation", - "id": "ae8f93b9-b1c5-4564-a8dd-882f69207ec3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.69, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ca7717d-07cf-4391-bfd5-d311ff630610", - "resource": { - "resourceType": "Observation", - "id": "1ca7717d-07cf-4391-bfd5-d311ff630610", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.23, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9cab820-08d5-4167-acd2-4473c30d15e9", - "resource": { - "resourceType": "Observation", - "id": "a9cab820-08d5-4167-acd2-4473c30d15e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 138.89, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e7e10938-6cc0-4069-9f82-aa983e806b19", - "resource": { - "resourceType": "Observation", - "id": "e7e10938-6cc0-4069-9f82-aa983e806b19", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 3.76, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d43a719-33ad-47a3-bf44-ebe624c4d77e", - "resource": { - "resourceType": "Observation", - "id": "0d43a719-33ad-47a3-bf44-ebe624c4d77e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 109.42, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:397a8bdc-fe8e-47ec-91a8-26da70f09819", - "resource": { - "resourceType": "Observation", - "id": "397a8bdc-fe8e-47ec-91a8-26da70f09819", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 21.34, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1fbb14d8-653e-496f-a2fa-da7db6cd15b8", - "resource": { - "resourceType": "Observation", - "id": "1fbb14d8-653e-496f-a2fa-da7db6cd15b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28fc3a4f-8686-46ab-acda-7f4b898399ee", - "resource": { - "resourceType": "Observation", - "id": "28fc3a4f-8686-46ab-acda-7f4b898399ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "valueQuantity": { - "value": 5.96, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25869f6e-53aa-40ff-9d9b-e9d0feada885", - "resource": { - "resourceType": "MedicationRequest", - "id": "25869f6e-53aa-40ff-9d9b-e9d0feada885", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "authoredOn": "2015-11-16T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:90c5f7c5-8f99-4ae3-a947-20c1d952aadf", - "resource": { - "resourceType": "Claim", - "id": "90c5f7c5-8f99-4ae3-a947-20c1d952aadf", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "created": "2015-11-16T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:25869f6e-53aa-40ff-9d9b-e9d0feada885" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7d2e7923-1783-4589-87bb-e01d1981cb03", - "resource": { - "resourceType": "Immunization", - "id": "7d2e7923-1783-4589-87bb-e01d1981cb03", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "occurrenceDateTime": "2015-11-16T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:00f0866e-f982-4001-ae1d-58f2c0c87779", - "resource": { - "resourceType": "DiagnosticReport", - "id": "00f0866e-f982-4001-ae1d-58f2c0c87779", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - }, - "effectiveDateTime": "2015-11-16T04:15:51-08:00", - "issued": "2015-11-16T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:65104e37-d772-42e9-ab0a-c793d779e36f", - "display": "Glucose" - }, - { - "reference": "urn:uuid:d14c63db-3889-4321-a2b9-2698d9c0e258", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:ae8f93b9-b1c5-4564-a8dd-882f69207ec3", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:1ca7717d-07cf-4391-bfd5-d311ff630610", - "display": "Calcium" - }, - { - "reference": "urn:uuid:a9cab820-08d5-4167-acd2-4473c30d15e9", - "display": "Sodium" - }, - { - "reference": "urn:uuid:e7e10938-6cc0-4069-9f82-aa983e806b19", - "display": "Potassium" - }, - { - "reference": "urn:uuid:0d43a719-33ad-47a3-bf44-ebe624c4d77e", - "display": "Chloride" - }, - { - "reference": "urn:uuid:397a8bdc-fe8e-47ec-91a8-26da70f09819", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:ba9a039d-0ef0-43c1-9810-0fff2c723f35", - "resource": { - "resourceType": "Claim", - "id": "ba9a039d-0ef0-43c1-9810-0fff2c723f35", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "created": "2015-11-16T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7d2e7923-1783-4589-87bb-e01d1981cb03" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:804fbaa8-0721-48a7-ae59-b20a07fc2905", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "804fbaa8-0721-48a7-ae59-b20a07fc2905", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ba9a039d-0ef0-43c1-9810-0fff2c723f35" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2015-11-16T04:30:51-08:00", - "end": "2016-11-16T04:30:51-08:00" - }, - "created": "2015-11-16T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ba9a039d-0ef0-43c1-9810-0fff2c723f35" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2583e8ac-f7b4-41b9-b9cb-e80a38d44a77" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2015-11-16T04:15:51-08:00", - "end": "2015-11-16T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d", - "resource": { - "resourceType": "Encounter", - "id": "aa56cec4-c00a-4454-a033-76b56580687d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:74e8aea0-2a43-4176-9e04-fd91e9613dd8", - "resource": { - "resourceType": "Claim", - "id": "74e8aea0-2a43-4176-9e04-fd91e9613dd8", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:30:51-08:00" - }, - "created": "2016-02-15T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:47c945ba-b7ba-4afa-9bf7-7f001fa90751", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "47c945ba-b7ba-4afa-9bf7-7f001fa90751", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "74e8aea0-2a43-4176-9e04-fd91e9613dd8" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-02-15T04:30:51-08:00", - "end": "2017-02-15T04:30:51-08:00" - }, - "created": "2016-02-15T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:74e8aea0-2a43-4176-9e04-fd91e9613dd8" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:aa56cec4-c00a-4454-a033-76b56580687d" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3", - "resource": { - "resourceType": "Encounter", - "id": "6765d1a2-4909-4a35-88b1-5f1eb690e7b3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-16T04:34:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-16T04:34:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:08755398-e73c-4c46-af1d-b1301f096815", - "resource": { - "resourceType": "Procedure", - "id": "08755398-e73c-4c46-af1d-b1301f096815", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" - }, - "performedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:34:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:fefee1b5-c6c1-4b6c-9a4b-9242043c93d0", - "resource": { - "resourceType": "Claim", - "id": "fefee1b5-c6c1-4b6c-9a4b-9242043c93d0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-16T04:34:51-08:00" - }, - "created": "2016-02-16T04:34:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:08755398-e73c-4c46-af1d-b1301f096815" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:074620ab-31a2-4434-8cca-e14950d0b4d1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "074620ab-31a2-4434-8cca-e14950d0b4d1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fefee1b5-c6c1-4b6c-9a4b-9242043c93d0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-02-16T04:34:51-08:00", - "end": "2017-02-16T04:34:51-08:00" - }, - "created": "2016-02-16T04:34:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:fefee1b5-c6c1-4b6c-9a4b-9242043c93d0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-16T04:34:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:6765d1a2-4909-4a35-88b1-5f1eb690e7b3" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-16T04:34:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68", - "resource": { - "resourceType": "Encounter", - "id": "1d625271-3a13-4eab-a560-641b010fab68", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:53:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:53:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:65458bc2-4bb3-49ac-bd7a-756719221cc1", - "resource": { - "resourceType": "Observation", - "id": "65458bc2-4bb3-49ac-bd7a-756719221cc1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "therapy", - "display": "therapy" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "88040-1", - "display": "Response to cancer treatment" - } - ], - "text": "Response to cancer treatment" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" - }, - "effectiveDateTime": "2016-02-15T04:15:51-08:00", - "issued": "2016-02-15T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385633008", - "display": "Improving (qualifier value)" - } - ], - "text": "Improving (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff415025-4fdc-4dad-95c7-c7e03d8ee7da", - "resource": { - "resourceType": "Procedure", - "id": "ff415025-4fdc-4dad-95c7-c7e03d8ee7da", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" - }, - "performedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:38:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:a0e401f7-f75d-4979-8eeb-6f64d42a2989", - "resource": { - "resourceType": "Claim", - "id": "a0e401f7-f75d-4979-8eeb-6f64d42a2989", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:53:51-08:00" - }, - "created": "2016-02-15T04:53:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ff415025-4fdc-4dad-95c7-c7e03d8ee7da" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 10988.12, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b810d0ee-155b-4966-b24f-3e1d78836faf", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b810d0ee-155b-4966-b24f-3e1d78836faf", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a0e401f7-f75d-4979-8eeb-6f64d42a2989" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-02-15T04:53:51-08:00", - "end": "2017-02-15T04:53:51-08:00" - }, - "created": "2016-02-15T04:53:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:a0e401f7-f75d-4979-8eeb-6f64d42a2989" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:53:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1d625271-3a13-4eab-a560-641b010fab68" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2016-02-15T04:15:51-08:00", - "end": "2016-02-15T04:53:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 10988.12, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2197.6240000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 8790.496000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 10988.12, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 10988.12, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 8790.496000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51", - "resource": { - "resourceType": "Encounter", - "id": "25239180-f721-4740-92d9-5431876d9b51", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-06-06T05:15:51-07:00", - "end": "2016-06-06T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-06-06T05:15:51-07:00", - "end": "2016-06-06T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7241d53f-c28d-40f6-87e8-9d13adb9e3c3", - "resource": { - "resourceType": "Claim", - "id": "7241d53f-c28d-40f6-87e8-9d13adb9e3c3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-06-06T05:15:51-07:00", - "end": "2016-06-06T05:30:51-07:00" - }, - "created": "2016-06-06T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a40eaa1d-53b2-4ced-9701-f0f0eebf44c9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a40eaa1d-53b2-4ced-9701-f0f0eebf44c9", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7241d53f-c28d-40f6-87e8-9d13adb9e3c3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-06-06T05:30:51-07:00", - "end": "2017-06-06T05:30:51-07:00" - }, - "created": "2016-06-06T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7241d53f-c28d-40f6-87e8-9d13adb9e3c3" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2016-06-06T05:15:51-07:00", - "end": "2016-06-06T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:25239180-f721-4740-92d9-5431876d9b51" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6", - "resource": { - "resourceType": "Encounter", - "id": "73b11689-39d9-4c07-92a2-fcd6f137d5d6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-08-01T05:15:51-07:00", - "end": "2016-08-01T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-08-01T05:15:51-07:00", - "end": "2016-08-01T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1", - "resource": { - "resourceType": "Condition", - "id": "3af34e51-849b-479a-83c6-ca3b9263a9f1", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" - }, - "onsetDateTime": "2016-08-01T05:15:51-07:00", - "abatementDateTime": "2016-08-08T05:15:51-07:00", - "recordedDate": "2016-08-01T05:15:51-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:bc0d8224-b2fc-403b-8dca-021565a22dfa", - "resource": { - "resourceType": "Claim", - "id": "bc0d8224-b2fc-403b-8dca-021565a22dfa", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-08-01T05:15:51-07:00", - "end": "2016-08-01T05:30:51-07:00" - }, - "created": "2016-08-01T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b8873578-29eb-4ab2-9695-ef618506e0fd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b8873578-29eb-4ab2-9695-ef618506e0fd", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bc0d8224-b2fc-403b-8dca-021565a22dfa" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-08-01T05:30:51-07:00", - "end": "2017-08-01T05:30:51-07:00" - }, - "created": "2016-08-01T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:bc0d8224-b2fc-403b-8dca-021565a22dfa" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3af34e51-849b-479a-83c6-ca3b9263a9f1" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2016-08-01T05:15:51-07:00", - "end": "2016-08-01T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:73b11689-39d9-4c07-92a2-fcd6f137d5d6" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "servicedPeriod": { - "start": "2016-08-01T05:15:51-07:00", - "end": "2016-08-01T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60", - "resource": { - "resourceType": "Encounter", - "id": "44a8b8e0-4109-40bb-969a-9a2d23b56c60", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-10-03T05:15:51-07:00", - "end": "2016-10-03T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2016-10-03T05:15:51-07:00", - "end": "2016-10-03T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:60f58125-977a-4d08-9be3-42f6fe8b8c47", - "resource": { - "resourceType": "Claim", - "id": "60f58125-977a-4d08-9be3-42f6fe8b8c47", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-10-03T05:15:51-07:00", - "end": "2016-10-03T05:30:51-07:00" - }, - "created": "2016-10-03T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9a675af5-4f3a-4c26-885a-1b72ccd1bead", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9a675af5-4f3a-4c26-885a-1b72ccd1bead", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "60f58125-977a-4d08-9be3-42f6fe8b8c47" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-10-03T05:30:51-07:00", - "end": "2017-10-03T05:30:51-07:00" - }, - "created": "2016-10-03T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:60f58125-977a-4d08-9be3-42f6fe8b8c47" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2016-10-03T05:15:51-07:00", - "end": "2016-10-03T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:44a8b8e0-4109-40bb-969a-9a2d23b56c60" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6", - "resource": { - "resourceType": "Encounter", - "id": "be05b886-37df-4c36-8bf4-7eab3e9445d6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f12d620d-10c3-4475-b22b-d623a62fc2de", - "resource": { - "resourceType": "Observation", - "id": "f12d620d-10c3-4475-b22b-d623a62fc2de", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f8f7b163-3c42-4424-83f3-d77dd0270661", - "resource": { - "resourceType": "Observation", - "id": "f8f7b163-3c42-4424-83f3-d77dd0270661", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9b1f42ff-1f95-40fb-8c68-2dd5e9e73e1f", - "resource": { - "resourceType": "Observation", - "id": "9b1f42ff-1f95-40fb-8c68-2dd5e9e73e1f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e971fcff-4d62-4fd0-bdb3-9d141040b0fa", - "resource": { - "resourceType": "Observation", - "id": "e971fcff-4d62-4fd0-bdb3-9d141040b0fa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:28fc98fb-2954-4288-b519-1ab918ca827d", - "resource": { - "resourceType": "Observation", - "id": "28fc98fb-2954-4288-b519-1ab918ca827d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 115, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:72542d1b-bdce-4d3a-b964-db3ee94d8e56", - "resource": { - "resourceType": "Observation", - "id": "72542d1b-bdce-4d3a-b964-db3ee94d8e56", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 82, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71f64330-92fa-47c4-b557-e15e2b2f6443", - "resource": { - "resourceType": "Observation", - "id": "71f64330-92fa-47c4-b557-e15e2b2f6443", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:46799724-940f-4725-94a3-46c4d00515d2", - "resource": { - "resourceType": "Observation", - "id": "46799724-940f-4725-94a3-46c4d00515d2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 92.39, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5827a700-4e2a-4107-8661-170a932d388a", - "resource": { - "resourceType": "Observation", - "id": "5827a700-4e2a-4107-8661-170a932d388a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 8.45, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f36fadcb-e8d2-4599-bde6-b08f1296d074", - "resource": { - "resourceType": "Observation", - "id": "f36fadcb-e8d2-4599-bde6-b08f1296d074", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.79, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e8e933d-db08-4965-953a-df40d7bbc79a", - "resource": { - "resourceType": "Observation", - "id": "7e8e933d-db08-4965-953a-df40d7bbc79a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.39, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92b787a5-30dc-4591-b549-89479c5bdce5", - "resource": { - "resourceType": "Observation", - "id": "92b787a5-30dc-4591-b549-89479c5bdce5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 139.34, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f009654-9ec6-4c18-be61-35c587764ab9", - "resource": { - "resourceType": "Observation", - "id": "1f009654-9ec6-4c18-be61-35c587764ab9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.93, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f118ef35-5f46-4ce5-b38b-82064b1cdf79", - "resource": { - "resourceType": "Observation", - "id": "f118ef35-5f46-4ce5-b38b-82064b1cdf79", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 106.58, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c07a523-7bb1-407f-a508-f1b29078f23d", - "resource": { - "resourceType": "Observation", - "id": "3c07a523-7bb1-407f-a508-f1b29078f23d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.72, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:09baf491-bdd4-431a-b345-92cbc6715dac", - "resource": { - "resourceType": "Observation", - "id": "09baf491-bdd4-431a-b345-92cbc6715dac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 197.98, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e0127580-1d18-49d9-af8d-aecec758772b", - "resource": { - "resourceType": "Observation", - "id": "e0127580-1d18-49d9-af8d-aecec758772b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 100.79, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3ac0f342-0954-4f2a-b788-d707473739e5", - "resource": { - "resourceType": "Observation", - "id": "3ac0f342-0954-4f2a-b788-d707473739e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 117.9, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bb68cecc-4923-48f6-abe8-d3021562bab1", - "resource": { - "resourceType": "Observation", - "id": "bb68cecc-4923-48f6-abe8-d3021562bab1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 59.93, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:58016733-a0f3-4e54-9c8e-3d4013b253e7", - "resource": { - "resourceType": "Observation", - "id": "58016733-a0f3-4e54-9c8e-3d4013b253e7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a3376cf5-4e9a-4426-bf87-704d6957cccc", - "resource": { - "resourceType": "Observation", - "id": "a3376cf5-4e9a-4426-bf87-704d6957cccc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.01, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f73f5a63-ab01-4f47-aa18-5a94efa4cdba", - "resource": { - "resourceType": "MedicationRequest", - "id": "f73f5a63-ab01-4f47-aa18-5a94efa4cdba", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "authoredOn": "2016-11-21T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a7ccd4d9-c1ef-4867-80b3-2448efb6801a", - "resource": { - "resourceType": "Claim", - "id": "a7ccd4d9-c1ef-4867-80b3-2448efb6801a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "created": "2016-11-21T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:f73f5a63-ab01-4f47-aa18-5a94efa4cdba" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d7c91657-fdf4-48c5-b0ab-78775a66d86f", - "resource": { - "resourceType": "Immunization", - "id": "d7c91657-fdf4-48c5-b0ab-78775a66d86f", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "occurrenceDateTime": "2016-11-21T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:3886cfff-344f-4a97-9bdc-b8f30c5302bc", - "resource": { - "resourceType": "DiagnosticReport", - "id": "3886cfff-344f-4a97-9bdc-b8f30c5302bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:46799724-940f-4725-94a3-46c4d00515d2", - "display": "Glucose" - }, - { - "reference": "urn:uuid:5827a700-4e2a-4107-8661-170a932d388a", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:f36fadcb-e8d2-4599-bde6-b08f1296d074", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:7e8e933d-db08-4965-953a-df40d7bbc79a", - "display": "Calcium" - }, - { - "reference": "urn:uuid:92b787a5-30dc-4591-b549-89479c5bdce5", - "display": "Sodium" - }, - { - "reference": "urn:uuid:1f009654-9ec6-4c18-be61-35c587764ab9", - "display": "Potassium" - }, - { - "reference": "urn:uuid:f118ef35-5f46-4ce5-b38b-82064b1cdf79", - "display": "Chloride" - }, - { - "reference": "urn:uuid:3c07a523-7bb1-407f-a508-f1b29078f23d", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:aa99e6d6-9cce-4d56-b3b3-4439ed011984", - "resource": { - "resourceType": "DiagnosticReport", - "id": "aa99e6d6-9cce-4d56-b3b3-4439ed011984", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - }, - "effectiveDateTime": "2016-11-21T04:15:51-08:00", - "issued": "2016-11-21T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:09baf491-bdd4-431a-b345-92cbc6715dac", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:e0127580-1d18-49d9-af8d-aecec758772b", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:3ac0f342-0954-4f2a-b788-d707473739e5", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:bb68cecc-4923-48f6-abe8-d3021562bab1", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7d45ec7c-d373-442b-aa09-7dd7fabcaabe", - "resource": { - "resourceType": "Claim", - "id": "7d45ec7c-d373-442b-aa09-7dd7fabcaabe", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "created": "2016-11-21T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d7c91657-fdf4-48c5-b0ab-78775a66d86f" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4e25f12e-53da-4b67-afa1-6220bf889e74", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4e25f12e-53da-4b67-afa1-6220bf889e74", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7d45ec7c-d373-442b-aa09-7dd7fabcaabe" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2016-11-21T04:30:51-08:00", - "end": "2017-11-21T04:30:51-08:00" - }, - "created": "2016-11-21T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7d45ec7c-d373-442b-aa09-7dd7fabcaabe" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:be05b886-37df-4c36-8bf4-7eab3e9445d6" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2016-11-21T04:15:51-08:00", - "end": "2016-11-21T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a", - "resource": { - "resourceType": "Encounter", - "id": "4d1849a6-ff4c-418b-bd72-b524d972018a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f44d668d-3307-4183-9e84-d4017e7d8c5f", - "resource": { - "resourceType": "Claim", - "id": "f44d668d-3307-4183-9e84-d4017e7d8c5f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:30:51-08:00" - }, - "created": "2017-01-23T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:791b968f-c30e-4d52-bb9a-ae25bd424dc0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "791b968f-c30e-4d52-bb9a-ae25bd424dc0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f44d668d-3307-4183-9e84-d4017e7d8c5f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-01-23T04:30:51-08:00", - "end": "2018-01-23T04:30:51-08:00" - }, - "created": "2017-01-23T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:f44d668d-3307-4183-9e84-d4017e7d8c5f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4d1849a6-ff4c-418b-bd72-b524d972018a" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40", - "resource": { - "resourceType": "Encounter", - "id": "c528e256-7619-4a5f-8384-48db8c7a1e40", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-24T04:38:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-24T04:38:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e0724ed8-53b9-4752-9840-9921e99271b3", - "resource": { - "resourceType": "Procedure", - "id": "e0724ed8-53b9-4752-9840-9921e99271b3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" - }, - "performedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:38:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:cd880169-5053-4258-bc4a-0b4189326536", - "resource": { - "resourceType": "Claim", - "id": "cd880169-5053-4258-bc4a-0b4189326536", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-24T04:38:51-08:00" - }, - "created": "2017-01-24T04:38:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e0724ed8-53b9-4752-9840-9921e99271b3" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:33713f2f-3760-4c70-8191-4efc896f963d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "33713f2f-3760-4c70-8191-4efc896f963d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cd880169-5053-4258-bc4a-0b4189326536" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-01-24T04:38:51-08:00", - "end": "2018-01-24T04:38:51-08:00" - }, - "created": "2017-01-24T04:38:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:cd880169-5053-4258-bc4a-0b4189326536" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-24T04:38:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c528e256-7619-4a5f-8384-48db8c7a1e40" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-24T04:38:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1", - "resource": { - "resourceType": "Encounter", - "id": "71cd8fee-25f0-44c6-8aa6-bb0031af72a1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:41:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:41:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:960d5783-e2e4-4692-a233-1233300e816b", - "resource": { - "resourceType": "Observation", - "id": "960d5783-e2e4-4692-a233-1233300e816b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "therapy", - "display": "therapy" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "88040-1", - "display": "Response to cancer treatment" - } - ], - "text": "Response to cancer treatment" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" - }, - "effectiveDateTime": "2017-01-23T04:15:51-08:00", - "issued": "2017-01-23T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385633008", - "display": "Improving (qualifier value)" - } - ], - "text": "Improving (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a524a6e8-00f8-41f2-b1c4-3238d2e7ed45", - "resource": { - "resourceType": "Procedure", - "id": "a524a6e8-00f8-41f2-b1c4-3238d2e7ed45", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" - }, - "performedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:26:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0cbd621b-20ec-4d1b-b955-4e8e3d7c8375", - "resource": { - "resourceType": "Claim", - "id": "0cbd621b-20ec-4d1b-b955-4e8e3d7c8375", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:41:51-08:00" - }, - "created": "2017-01-23T04:41:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a524a6e8-00f8-41f2-b1c4-3238d2e7ed45" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 10282.89, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:03603520-d43f-479e-86be-5926a6114970", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "03603520-d43f-479e-86be-5926a6114970", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0cbd621b-20ec-4d1b-b955-4e8e3d7c8375" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-01-23T04:41:51-08:00", - "end": "2018-01-23T04:41:51-08:00" - }, - "created": "2017-01-23T04:41:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0cbd621b-20ec-4d1b-b955-4e8e3d7c8375" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:41:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:71cd8fee-25f0-44c6-8aa6-bb0031af72a1" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2017-01-23T04:15:51-08:00", - "end": "2017-01-23T04:41:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 10282.89, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2056.578, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 8226.312, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 10282.89, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 10282.89, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 8226.312, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8", - "resource": { - "resourceType": "Encounter", - "id": "921a28d6-913a-472c-88a7-dfb9b45895e8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-05-13T05:15:51-07:00", - "end": "2017-05-13T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-05-13T05:15:51-07:00", - "end": "2017-05-13T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29", - "resource": { - "resourceType": "Condition", - "id": "7aeb9bc7-24f9-4145-b5b3-15be17203a29", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" - }, - "onsetDateTime": "2017-05-13T05:15:51-07:00", - "abatementDateTime": "2017-06-03T05:15:51-07:00", - "recordedDate": "2017-05-13T05:15:51-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b5b01012-a227-41dc-89f5-e2485647ece0", - "resource": { - "resourceType": "Claim", - "id": "b5b01012-a227-41dc-89f5-e2485647ece0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-05-13T05:15:51-07:00", - "end": "2017-05-13T05:30:51-07:00" - }, - "created": "2017-05-13T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fe34960b-1bf7-473e-857a-d644fb405805", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "fe34960b-1bf7-473e-857a-d644fb405805", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b5b01012-a227-41dc-89f5-e2485647ece0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-05-13T05:30:51-07:00", - "end": "2018-05-13T05:30:51-07:00" - }, - "created": "2017-05-13T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b5b01012-a227-41dc-89f5-e2485647ece0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7aeb9bc7-24f9-4145-b5b3-15be17203a29" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2017-05-13T05:15:51-07:00", - "end": "2017-05-13T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:921a28d6-913a-472c-88a7-dfb9b45895e8" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "servicedPeriod": { - "start": "2017-05-13T05:15:51-07:00", - "end": "2017-05-13T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf", - "resource": { - "resourceType": "Encounter", - "id": "9d6e40c7-408e-4375-8830-e0b61faa8baf", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-05-22T05:15:51-07:00", - "end": "2017-05-22T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-05-22T05:15:51-07:00", - "end": "2017-05-22T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c0eb680d-2f42-445d-b3a5-519d5c1fa3cc", - "resource": { - "resourceType": "Claim", - "id": "c0eb680d-2f42-445d-b3a5-519d5c1fa3cc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-05-22T05:15:51-07:00", - "end": "2017-05-22T05:30:51-07:00" - }, - "created": "2017-05-22T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b610b97f-954f-4a50-8036-51c8557ce56b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b610b97f-954f-4a50-8036-51c8557ce56b", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c0eb680d-2f42-445d-b3a5-519d5c1fa3cc" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-05-22T05:30:51-07:00", - "end": "2018-05-22T05:30:51-07:00" - }, - "created": "2017-05-22T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:c0eb680d-2f42-445d-b3a5-519d5c1fa3cc" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-05-22T05:15:51-07:00", - "end": "2017-05-22T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:9d6e40c7-408e-4375-8830-e0b61faa8baf" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb", - "resource": { - "resourceType": "Encounter", - "id": "a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-08-25T05:15:51-07:00", - "end": "2017-08-25T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-08-25T05:15:51-07:00", - "end": "2017-08-25T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488", - "resource": { - "resourceType": "Condition", - "id": "e086a3ee-b70c-4698-9d6d-0de40fa81488", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" - }, - "onsetDateTime": "2017-08-25T05:15:51-07:00", - "abatementDateTime": "2017-09-06T05:15:51-07:00", - "recordedDate": "2017-08-25T05:15:51-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:8ca17759-9c08-4f83-93a8-5e2346524b2a", - "resource": { - "resourceType": "Observation", - "id": "8ca17759-9c08-4f83-93a8-5e2346524b2a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - }, - { - "system": "http://loinc.org", - "code": "8331-1", - "display": "Oral temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" - }, - "effectiveDateTime": "2017-08-25T05:15:51-07:00", - "issued": "2017-08-25T05:15:51.927-07:00", - "valueQuantity": { - "value": 37.861, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd8448c1-d0ef-482d-a44a-33976a57dbac", - "resource": { - "resourceType": "Claim", - "id": "dd8448c1-d0ef-482d-a44a-33976a57dbac", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-08-25T05:15:51-07:00", - "end": "2017-08-25T05:30:51-07:00" - }, - "created": "2017-08-25T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2d081651-0596-4aa6-b209-8935ae2c7e19", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2d081651-0596-4aa6-b209-8935ae2c7e19", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "dd8448c1-d0ef-482d-a44a-33976a57dbac" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-08-25T05:30:51-07:00", - "end": "2018-08-25T05:30:51-07:00" - }, - "created": "2017-08-25T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:dd8448c1-d0ef-482d-a44a-33976a57dbac" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:e086a3ee-b70c-4698-9d6d-0de40fa81488" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2017-08-25T05:15:51-07:00", - "end": "2017-08-25T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a0136e8c-6cc9-4c09-8bd4-581e0b73fdbb" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "servicedPeriod": { - "start": "2017-08-25T05:15:51-07:00", - "end": "2017-08-25T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246", - "resource": { - "resourceType": "Encounter", - "id": "25cffaa1-5504-429d-b23c-396d0c723246", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-09-04T05:15:51-07:00", - "end": "2017-09-04T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-09-04T05:15:51-07:00", - "end": "2017-09-04T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5b562284-6d0c-409f-bfe6-290e5d6eab6a", - "resource": { - "resourceType": "Claim", - "id": "5b562284-6d0c-409f-bfe6-290e5d6eab6a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-09-04T05:15:51-07:00", - "end": "2017-09-04T05:30:51-07:00" - }, - "created": "2017-09-04T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5da29a20-e52a-4e25-90ad-71e3e42209a0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5da29a20-e52a-4e25-90ad-71e3e42209a0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5b562284-6d0c-409f-bfe6-290e5d6eab6a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-09-04T05:30:51-07:00", - "end": "2018-09-04T05:30:51-07:00" - }, - "created": "2017-09-04T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5b562284-6d0c-409f-bfe6-290e5d6eab6a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-09-04T05:15:51-07:00", - "end": "2017-09-04T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:25cffaa1-5504-429d-b23c-396d0c723246" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652", - "resource": { - "resourceType": "Encounter", - "id": "4fcfb59a-0a02-457e-ac45-d3a1c564f652", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:77e9073d-f4d8-4074-a8d5-bbae0463c764", - "resource": { - "resourceType": "Observation", - "id": "77e9073d-f4d8-4074-a8d5-bbae0463c764", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99a8dd5e-deda-4d0b-bf24-c9a8e106ced7", - "resource": { - "resourceType": "Observation", - "id": "99a8dd5e-deda-4d0b-bf24-c9a8e106ced7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7734cb03-d5fd-496b-a840-c0e8fe85a044", - "resource": { - "resourceType": "Observation", - "id": "7734cb03-d5fd-496b-a840-c0e8fe85a044", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:038ea4d5-4c60-4e83-a41f-0d9405e68f2a", - "resource": { - "resourceType": "Observation", - "id": "038ea4d5-4c60-4e83-a41f-0d9405e68f2a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ec9e025-406d-4d43-a7aa-4f66e79646a0", - "resource": { - "resourceType": "Observation", - "id": "0ec9e025-406d-4d43-a7aa-4f66e79646a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 87, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 112, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7766f97-882d-4d06-aa97-a4e081fe66df", - "resource": { - "resourceType": "Observation", - "id": "a7766f97-882d-4d06-aa97-a4e081fe66df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bba53133-ea29-44fa-bb1c-602ecfc7c47e", - "resource": { - "resourceType": "Observation", - "id": "bba53133-ea29-44fa-bb1c-602ecfc7c47e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11bfc228-f8eb-432a-8e85-5409e878e081", - "resource": { - "resourceType": "Observation", - "id": "11bfc228-f8eb-432a-8e85-5409e878e081", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 67.48, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a13f06da-9099-429a-921b-e068b358985a", - "resource": { - "resourceType": "Observation", - "id": "a13f06da-9099-429a-921b-e068b358985a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 12.39, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d379bd6a-cea1-4df3-ade1-c632ba84c235", - "resource": { - "resourceType": "Observation", - "id": "d379bd6a-cea1-4df3-ade1-c632ba84c235", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.78, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:abfa4065-b023-4821-a2a1-ddb68dcb2e7a", - "resource": { - "resourceType": "Observation", - "id": "abfa4065-b023-4821-a2a1-ddb68dcb2e7a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.47, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b13e310-cbb1-4073-83b7-0b14c5210ef4", - "resource": { - "resourceType": "Observation", - "id": "6b13e310-cbb1-4073-83b7-0b14c5210ef4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 142.01, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba524569-514a-4172-87a4-158328c244e6", - "resource": { - "resourceType": "Observation", - "id": "ba524569-514a-4172-87a4-158328c244e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.23, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25d75864-1daa-4b87-a322-7a888027eba3", - "resource": { - "resourceType": "Observation", - "id": "25d75864-1daa-4b87-a322-7a888027eba3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 108.44, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11719ea0-6348-4c70-9574-61da59491254", - "resource": { - "resourceType": "Observation", - "id": "11719ea0-6348-4c70-9574-61da59491254", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.18, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7f9d0f76-b695-4272-a365-7101faf9fc28", - "resource": { - "resourceType": "Observation", - "id": "7f9d0f76-b695-4272-a365-7101faf9fc28", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82731ca9-5a30-4b7c-b8ea-0214e78ae61a", - "resource": { - "resourceType": "Observation", - "id": "82731ca9-5a30-4b7c-b8ea-0214e78ae61a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.26, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ec7c769d-e8d1-47ad-8919-6c1f568e9499", - "resource": { - "resourceType": "MedicationRequest", - "id": "ec7c769d-e8d1-47ad-8919-6c1f568e9499", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "authoredOn": "2017-11-27T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:15b3b06e-717c-443f-a1af-695d9d17f765", - "resource": { - "resourceType": "Claim", - "id": "15b3b06e-717c-443f-a1af-695d9d17f765", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "created": "2017-11-27T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ec7c769d-e8d1-47ad-8919-6c1f568e9499" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7156756a-5620-4e70-b0e3-3b73fe5fec8f", - "resource": { - "resourceType": "Immunization", - "id": "7156756a-5620-4e70-b0e3-3b73fe5fec8f", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "occurrenceDateTime": "2017-11-27T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:5075e05d-7aff-413c-b816-dfa3ead366f3", - "resource": { - "resourceType": "DiagnosticReport", - "id": "5075e05d-7aff-413c-b816-dfa3ead366f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - }, - "effectiveDateTime": "2017-11-27T04:15:51-08:00", - "issued": "2017-11-27T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:11bfc228-f8eb-432a-8e85-5409e878e081", - "display": "Glucose" - }, - { - "reference": "urn:uuid:a13f06da-9099-429a-921b-e068b358985a", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:d379bd6a-cea1-4df3-ade1-c632ba84c235", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:abfa4065-b023-4821-a2a1-ddb68dcb2e7a", - "display": "Calcium" - }, - { - "reference": "urn:uuid:6b13e310-cbb1-4073-83b7-0b14c5210ef4", - "display": "Sodium" - }, - { - "reference": "urn:uuid:ba524569-514a-4172-87a4-158328c244e6", - "display": "Potassium" - }, - { - "reference": "urn:uuid:25d75864-1daa-4b87-a322-7a888027eba3", - "display": "Chloride" - }, - { - "reference": "urn:uuid:11719ea0-6348-4c70-9574-61da59491254", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e", - "resource": { - "resourceType": "Claim", - "id": "9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "created": "2017-11-27T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7156756a-5620-4e70-b0e3-3b73fe5fec8f" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4dfb468f-701e-499b-b17f-1220670ac9f0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4dfb468f-701e-499b-b17f-1220670ac9f0", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-11-27T04:30:51-08:00", - "end": "2018-11-27T04:30:51-08:00" - }, - "created": "2017-11-27T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9f29d9ab-e9a8-40c3-9b0d-1ea7229ea98e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4fcfb59a-0a02-457e-ac45-d3a1c564f652" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2017-11-27T04:15:51-08:00", - "end": "2017-11-27T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee", - "resource": { - "resourceType": "Encounter", - "id": "69459341-de89-4353-8c45-b61275db24ee", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:86d8ad2b-e60b-427c-903e-2804c948e723", - "resource": { - "resourceType": "Claim", - "id": "86d8ad2b-e60b-427c-903e-2804c948e723", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:30:51-08:00" - }, - "created": "2017-12-25T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:52f2d5e5-2e85-4829-a6fa-9e36bc728da3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "52f2d5e5-2e85-4829-a6fa-9e36bc728da3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "86d8ad2b-e60b-427c-903e-2804c948e723" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-12-25T04:30:51-08:00", - "end": "2018-12-25T04:30:51-08:00" - }, - "created": "2017-12-25T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:86d8ad2b-e60b-427c-903e-2804c948e723" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:69459341-de89-4353-8c45-b61275db24ee" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea", - "resource": { - "resourceType": "Encounter", - "id": "defaa09a-4bd0-4e24-a7e8-97d0a0db72ea", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-26T04:38:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-26T04:38:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:af2d564d-b557-418b-bbd2-d510d10d21f1", - "resource": { - "resourceType": "Procedure", - "id": "af2d564d-b557-418b-bbd2-d510d10d21f1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" - }, - "performedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:38:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b232f88e-b5c8-4287-8499-164d5364f144", - "resource": { - "resourceType": "Claim", - "id": "b232f88e-b5c8-4287-8499-164d5364f144", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-26T04:38:51-08:00" - }, - "created": "2017-12-26T04:38:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:af2d564d-b557-418b-bbd2-d510d10d21f1" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4fc74029-6198-448c-b038-055b51276e75", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4fc74029-6198-448c-b038-055b51276e75", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b232f88e-b5c8-4287-8499-164d5364f144" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-12-26T04:38:51-08:00", - "end": "2018-12-26T04:38:51-08:00" - }, - "created": "2017-12-26T04:38:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b232f88e-b5c8-4287-8499-164d5364f144" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-26T04:38:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:defaa09a-4bd0-4e24-a7e8-97d0a0db72ea" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-26T04:38:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3", - "resource": { - "resourceType": "Encounter", - "id": "ef1e49b1-1818-47f4-8f77-b620a6747bc3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:40:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:40:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c81a4b93-cdad-49cd-a4c2-a5b209da107c", - "resource": { - "resourceType": "Observation", - "id": "c81a4b93-cdad-49cd-a4c2-a5b209da107c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "therapy", - "display": "therapy" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "88040-1", - "display": "Response to cancer treatment" - } - ], - "text": "Response to cancer treatment" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" - }, - "effectiveDateTime": "2017-12-25T04:15:51-08:00", - "issued": "2017-12-25T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385633008", - "display": "Improving (qualifier value)" - } - ], - "text": "Improving (qualifier value)" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba020b52-384f-4119-afa3-bcca171c9333", - "resource": { - "resourceType": "Procedure", - "id": "ba020b52-384f-4119-afa3-bcca171c9333", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" - }, - "performedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:25:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e1963125-2416-46aa-9612-fb0d496d3732", - "resource": { - "resourceType": "Claim", - "id": "e1963125-2416-46aa-9612-fb0d496d3732", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:40:51-08:00" - }, - "created": "2017-12-25T04:40:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ba020b52-384f-4119-afa3-bcca171c9333" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 11061.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:42008aa2-f532-4c51-ab91-99a808ffbc17", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "42008aa2-f532-4c51-ab91-99a808ffbc17", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e1963125-2416-46aa-9612-fb0d496d3732" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2017-12-25T04:40:51-08:00", - "end": "2018-12-25T04:40:51-08:00" - }, - "created": "2017-12-25T04:40:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e1963125-2416-46aa-9612-fb0d496d3732" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:40:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ef1e49b1-1818-47f4-8f77-b620a6747bc3" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2017-12-25T04:15:51-08:00", - "end": "2017-12-25T04:40:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 11061.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2212.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 8849.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11061.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11061.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 8849.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733", - "resource": { - "resourceType": "Encounter", - "id": "0653ded2-d1f8-49db-a068-f0a35c96c733", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-04-16T05:15:51-07:00", - "end": "2018-04-16T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2018-04-16T05:15:51-07:00", - "end": "2018-04-16T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4243e264-2329-4726-9ac8-369d6bde35bc", - "resource": { - "resourceType": "Claim", - "id": "4243e264-2329-4726-9ac8-369d6bde35bc", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-04-16T05:15:51-07:00", - "end": "2018-04-16T05:30:51-07:00" - }, - "created": "2018-04-16T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a4a52946-7b40-428c-acbd-3a320da54164", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a4a52946-7b40-428c-acbd-3a320da54164", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4243e264-2329-4726-9ac8-369d6bde35bc" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-04-16T05:30:51-07:00", - "end": "2019-04-16T05:30:51-07:00" - }, - "created": "2018-04-16T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:4243e264-2329-4726-9ac8-369d6bde35bc" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2018-04-16T05:15:51-07:00", - "end": "2018-04-16T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0653ded2-d1f8-49db-a068-f0a35c96c733" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a", - "resource": { - "resourceType": "Encounter", - "id": "f8a5cf23-c2b5-40e1-ac08-482b606c7f3a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-08-13T05:15:51-07:00", - "end": "2018-08-13T05:30:51-07:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2018-08-13T05:15:51-07:00", - "end": "2018-08-13T05:30:51-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fbfbb74e-0420-474e-a434-d4db67cc6caf", - "resource": { - "resourceType": "Claim", - "id": "fbfbb74e-0420-474e-a434-d4db67cc6caf", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-08-13T05:15:51-07:00", - "end": "2018-08-13T05:30:51-07:00" - }, - "created": "2018-08-13T05:30:51-07:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:405f7699-9c13-46f7-95bb-380d58969a92", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "405f7699-9c13-46f7-95bb-380d58969a92", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fbfbb74e-0420-474e-a434-d4db67cc6caf" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-08-13T05:30:51-07:00", - "end": "2019-08-13T05:30:51-07:00" - }, - "created": "2018-08-13T05:30:51-07:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:fbfbb74e-0420-474e-a434-d4db67cc6caf" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2018-08-13T05:15:51-07:00", - "end": "2018-08-13T05:30:51-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f8a5cf23-c2b5-40e1-ac08-482b606c7f3a" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8", - "resource": { - "resourceType": "Encounter", - "id": "a16a6f17-85b4-45d0-a5a4-d7fb05f508d8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:402eb990-0cef-4cce-9517-9b431a941424", - "resource": { - "resourceType": "Observation", - "id": "402eb990-0cef-4cce-9517-9b431a941424", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0485dc4a-d71b-4085-bc71-cdb343a55f0c", - "resource": { - "resourceType": "Observation", - "id": "0485dc4a-d71b-4085-bc71-cdb343a55f0c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2fa854b4-4bfb-46fe-80e1-9eda5a7cc477", - "resource": { - "resourceType": "Observation", - "id": "2fa854b4-4bfb-46fe-80e1-9eda5a7cc477", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c0d8cdd-edec-40b4-85fc-e396ea3deabb", - "resource": { - "resourceType": "Observation", - "id": "9c0d8cdd-edec-40b4-85fc-e396ea3deabb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:57dda705-a066-4cd2-a415-893aedc2bd41", - "resource": { - "resourceType": "Observation", - "id": "57dda705-a066-4cd2-a415-893aedc2bd41", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 118, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b04bc8fa-ca0d-4eb5-ab62-9c42899e05e3", - "resource": { - "resourceType": "Observation", - "id": "b04bc8fa-ca0d-4eb5-ab62-9c42899e05e3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 86, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:661e4067-649d-4a37-8f02-50d89b47e3f4", - "resource": { - "resourceType": "Observation", - "id": "661e4067-649d-4a37-8f02-50d89b47e3f4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca1df88e-e196-4189-8710-d68b85ade4cd", - "resource": { - "resourceType": "Observation", - "id": "ca1df88e-e196-4189-8710-d68b85ade4cd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 65.59, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9cb10ee-a277-456f-9a67-006a4da59bbf", - "resource": { - "resourceType": "Observation", - "id": "c9cb10ee-a277-456f-9a67-006a4da59bbf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 13.13, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0d36548a-7da4-45df-bf21-fc52e82389bb", - "resource": { - "resourceType": "Observation", - "id": "0d36548a-7da4-45df-bf21-fc52e82389bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.66, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fad38a42-303d-4df9-aaca-d0249af1bc35", - "resource": { - "resourceType": "Observation", - "id": "fad38a42-303d-4df9-aaca-d0249af1bc35", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.04, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dcdfe68f-5eb3-4653-9442-ba70b9a38551", - "resource": { - "resourceType": "Observation", - "id": "dcdfe68f-5eb3-4653-9442-ba70b9a38551", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 139.01, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba5d8607-99df-48dc-a201-a768fa5ee017", - "resource": { - "resourceType": "Observation", - "id": "ba5d8607-99df-48dc-a201-a768fa5ee017", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.07, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4efeccf-3e61-4874-b7ad-8784faa4271f", - "resource": { - "resourceType": "Observation", - "id": "c4efeccf-3e61-4874-b7ad-8784faa4271f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 102.09, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4893f495-9876-40ce-b506-5bbcb46ec280", - "resource": { - "resourceType": "Observation", - "id": "4893f495-9876-40ce-b506-5bbcb46ec280", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 23.05, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69b420d1-9a5a-443b-88bf-0c398faec3a9", - "resource": { - "resourceType": "Observation", - "id": "69b420d1-9a5a-443b-88bf-0c398faec3a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 5.9101, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2be60dc0-d024-4db5-81cb-c582dc15a488", - "resource": { - "resourceType": "Observation", - "id": "2be60dc0-d024-4db5-81cb-c582dc15a488", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 4.6104, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4ae9dd3-d629-4f69-8723-0a51262d5f28", - "resource": { - "resourceType": "Observation", - "id": "c4ae9dd3-d629-4f69-8723-0a51262d5f28", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 16.695, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:244d7604-018b-41d8-a10b-4f200ee438c1", - "resource": { - "resourceType": "Observation", - "id": "244d7604-018b-41d8-a10b-4f200ee438c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 35.366, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:da104bff-2240-49ef-8ed9-eb25010c6f08", - "resource": { - "resourceType": "Observation", - "id": "da104bff-2240-49ef-8ed9-eb25010c6f08", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 90.04, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29fbf375-fe82-466c-acbc-12559dad693c", - "resource": { - "resourceType": "Observation", - "id": "29fbf375-fe82-466c-acbc-12559dad693c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.376, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6a59543c-d6ae-47d8-b519-4abf27ea3295", - "resource": { - "resourceType": "Observation", - "id": "6a59543c-d6ae-47d8-b519-4abf27ea3295", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 35.927, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:09aaec1f-0e02-4c07-a3ac-8b383535b013", - "resource": { - "resourceType": "Observation", - "id": "09aaec1f-0e02-4c07-a3ac-8b383535b013", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 44.427, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e20a3a4-1529-43d4-9196-c6d78cedada6", - "resource": { - "resourceType": "Observation", - "id": "4e20a3a4-1529-43d4-9196-c6d78cedada6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 412.82, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:698b6782-257a-4ed4-807e-d8dc04cc8614", - "resource": { - "resourceType": "Observation", - "id": "698b6782-257a-4ed4-807e-d8dc04cc8614", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 316.13, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d6e6775c-d29f-4152-8d76-27ef115a8248", - "resource": { - "resourceType": "Observation", - "id": "d6e6775c-d29f-4152-8d76-27ef115a8248", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 10.306, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:899c5e79-8be4-4f04-a844-c34f190a81cf", - "resource": { - "resourceType": "Observation", - "id": "899c5e79-8be4-4f04-a844-c34f190a81cf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc9babc9-368e-47e5-9b2d-0f4326b9a209", - "resource": { - "resourceType": "Observation", - "id": "fc9babc9-368e-47e5-9b2d-0f4326b9a209", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.38, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf", - "resource": { - "resourceType": "Procedure", - "id": "d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "performedPeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:30:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:45fce3e2-80c4-4fc3-8719-0fae8df44547", - "resource": { - "resourceType": "MedicationRequest", - "id": "45fce3e2-80c4-4fc3-8719-0fae8df44547", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "authoredOn": "2018-12-03T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ddefe41a-1ca4-4da2-966d-717099da18cd", - "resource": { - "resourceType": "Claim", - "id": "ddefe41a-1ca4-4da2-966d-717099da18cd", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "created": "2018-12-03T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:45fce3e2-80c4-4fc3-8719-0fae8df44547" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d3e0de66-678d-448d-9374-fa13288a2bec", - "resource": { - "resourceType": "Immunization", - "id": "d3e0de66-678d-448d-9374-fa13288a2bec", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "occurrenceDateTime": "2018-12-03T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:268fead9-179f-414f-8452-0a9261f9c790", - "resource": { - "resourceType": "Immunization", - "id": "268fead9-179f-414f-8452-0a9261f9c790", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "occurrenceDateTime": "2018-12-03T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c99c9327-f69f-4057-886e-f75ff2a8dfb8", - "resource": { - "resourceType": "DiagnosticReport", - "id": "c99c9327-f69f-4057-886e-f75ff2a8dfb8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:ca1df88e-e196-4189-8710-d68b85ade4cd", - "display": "Glucose" - }, - { - "reference": "urn:uuid:c9cb10ee-a277-456f-9a67-006a4da59bbf", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:0d36548a-7da4-45df-bf21-fc52e82389bb", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:fad38a42-303d-4df9-aaca-d0249af1bc35", - "display": "Calcium" - }, - { - "reference": "urn:uuid:dcdfe68f-5eb3-4653-9442-ba70b9a38551", - "display": "Sodium" - }, - { - "reference": "urn:uuid:ba5d8607-99df-48dc-a201-a768fa5ee017", - "display": "Potassium" - }, - { - "reference": "urn:uuid:c4efeccf-3e61-4874-b7ad-8784faa4271f", - "display": "Chloride" - }, - { - "reference": "urn:uuid:4893f495-9876-40ce-b506-5bbcb46ec280", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:487a8a4b-8df5-45a1-90e2-3d7844ee4fc5", - "resource": { - "resourceType": "DiagnosticReport", - "id": "487a8a4b-8df5-45a1-90e2-3d7844ee4fc5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - }, - "effectiveDateTime": "2018-12-03T04:15:51-08:00", - "issued": "2018-12-03T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:69b420d1-9a5a-443b-88bf-0c398faec3a9", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:2be60dc0-d024-4db5-81cb-c582dc15a488", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:c4ae9dd3-d629-4f69-8723-0a51262d5f28", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:244d7604-018b-41d8-a10b-4f200ee438c1", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:da104bff-2240-49ef-8ed9-eb25010c6f08", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:29fbf375-fe82-466c-acbc-12559dad693c", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:6a59543c-d6ae-47d8-b519-4abf27ea3295", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:09aaec1f-0e02-4c07-a3ac-8b383535b013", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:4e20a3a4-1529-43d4-9196-c6d78cedada6", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:698b6782-257a-4ed4-807e-d8dc04cc8614", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:d6e6775c-d29f-4152-8d76-27ef115a8248", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:249cbe35-b846-406d-bf40-79838cb34695", - "resource": { - "resourceType": "Claim", - "id": "249cbe35-b846-406d-bf40-79838cb34695", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "created": "2018-12-03T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d3e0de66-678d-448d-9374-fa13288a2bec" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:268fead9-179f-414f-8452-0a9261f9c790" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:d7d1edad-9ce6-4ea1-9bc5-eeec52111fcf" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 690.58, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:edf36a35-d061-4442-b136-cb08d2bdd6ff", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "edf36a35-d061-4442-b136-cb08d2bdd6ff", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "249cbe35-b846-406d-bf40-79838cb34695" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-12-03T04:45:51-08:00", - "end": "2019-12-03T04:45:51-08:00" - }, - "created": "2018-12-03T04:45:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:249cbe35-b846-406d-bf40-79838cb34695" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:a16a6f17-85b4-45d0-a5a4-d7fb05f508d8" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "servicedPeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-03T04:15:51-08:00", - "end": "2018-12-03T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 690.58, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 138.116, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 552.464, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 690.58, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 690.58, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 777.296, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9", - "resource": { - "resourceType": "Encounter", - "id": "59cfb6a4-1794-48c6-8a71-7c0e405415e9", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cfae25a3-fed2-4f3d-b444-1c9a9562df4c", - "resource": { - "resourceType": "Claim", - "id": "cfae25a3-fed2-4f3d-b444-1c9a9562df4c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:30:51-08:00" - }, - "created": "2018-12-10T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:deb68b0a-0052-4720-950d-ea9a78dbdb99", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "deb68b0a-0052-4720-950d-ea9a78dbdb99", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cfae25a3-fed2-4f3d-b444-1c9a9562df4c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-12-10T04:30:51-08:00", - "end": "2019-12-10T04:30:51-08:00" - }, - "created": "2018-12-10T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:cfae25a3-fed2-4f3d-b444-1c9a9562df4c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:59cfb6a4-1794-48c6-8a71-7c0e405415e9" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d", - "resource": { - "resourceType": "Encounter", - "id": "ac9a985e-dfc3-4315-ad32-4a340be7594d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-11T04:32:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-11T04:32:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:02fb4cc4-ca2a-42d2-9500-ffe141963f19", - "resource": { - "resourceType": "Procedure", - "id": "02fb4cc4-ca2a-42d2-9500-ffe141963f19", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" - }, - "performedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:32:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:0962c5c9-7415-4d7c-859e-240f1608cf24", - "resource": { - "resourceType": "Claim", - "id": "0962c5c9-7415-4d7c-859e-240f1608cf24", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-11T04:32:51-08:00" - }, - "created": "2018-12-11T04:32:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:02fb4cc4-ca2a-42d2-9500-ffe141963f19" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:deb19e37-b06d-4fe4-83f4-338b35237769", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "deb19e37-b06d-4fe4-83f4-338b35237769", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0962c5c9-7415-4d7c-859e-240f1608cf24" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-12-11T04:32:51-08:00", - "end": "2019-12-11T04:32:51-08:00" - }, - "created": "2018-12-11T04:32:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:0962c5c9-7415-4d7c-859e-240f1608cf24" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-11T04:32:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ac9a985e-dfc3-4315-ad32-4a340be7594d" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-11T04:32:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10", - "resource": { - "resourceType": "Encounter", - "id": "6b91b276-e281-4052-99e0-ca11a5ad0b10", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:46:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:46:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a", - "resource": { - "resourceType": "Procedure", - "id": "07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" - }, - "performedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:31:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:db5c9b6f-cb0a-465e-a769-7eff0513f754", - "resource": { - "resourceType": "Claim", - "id": "db5c9b6f-cb0a-465e-a769-7eff0513f754", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:46:51-08:00" - }, - "created": "2018-12-10T04:46:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:07a202e8-ea6c-45bf-8e6d-b3ed2e6ad96a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 8977.80, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d3fa5207-39fb-4511-b870-c3a524d0d3d1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d3fa5207-39fb-4511-b870-c3a524d0d3d1", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "db5c9b6f-cb0a-465e-a769-7eff0513f754" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2018-12-10T04:46:51-08:00", - "end": "2019-12-10T04:46:51-08:00" - }, - "created": "2018-12-10T04:46:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:db5c9b6f-cb0a-465e-a769-7eff0513f754" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:46:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:6b91b276-e281-4052-99e0-ca11a5ad0b10" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2018-12-10T04:15:51-08:00", - "end": "2018-12-10T04:46:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 8977.80, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1795.56, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7182.24, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 8977.80, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 8977.80, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7182.24, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4", - "resource": { - "resourceType": "Encounter", - "id": "02d2ed94-8498-4317-bde6-15685b3922c4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "individual": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - } - } - ], - "period": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:dbe5f86c-7242-46f4-8e9a-a196be32876b", - "resource": { - "resourceType": "Observation", - "id": "dbe5f86c-7242-46f4-8e9a-a196be32876b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 172.4, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ef99c4b-c395-47cd-a493-f3298ca1544b", - "resource": { - "resourceType": "Observation", - "id": "6ef99c4b-c395-47cd-a493-f3298ca1544b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cfad61ac-d7fa-4bf4-8824-0cfc6b995e93", - "resource": { - "resourceType": "Observation", - "id": "cfad61ac-d7fa-4bf4-8824-0cfc6b995e93", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 81.3, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c11cd27-1730-4d64-9139-fcba691154da", - "resource": { - "resourceType": "Observation", - "id": "4c11cd27-1730-4d64-9139-fcba691154da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 27.35, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4ed5825-b78b-4546-b343-ba33bd378d8e", - "resource": { - "resourceType": "Observation", - "id": "c4ed5825-b78b-4546-b343-ba33bd378d8e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 137, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bf2a3404-9340-42ef-9701-4c78f48a538d", - "resource": { - "resourceType": "Observation", - "id": "bf2a3404-9340-42ef-9701-4c78f48a538d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1ade4a8-d866-4aa3-bd64-5a86c3fcc67b", - "resource": { - "resourceType": "Observation", - "id": "f1ade4a8-d866-4aa3-bd64-5a86c3fcc67b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", - "resource": { - "resourceType": "Observation", - "id": "9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2339-0", - "display": "Glucose" - } - ], - "text": "Glucose" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 91.67, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", - "resource": { - "resourceType": "Observation", - "id": "765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6299-2", - "display": "Urea Nitrogen" - } - ], - "text": "Urea Nitrogen" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 10.39, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3f0a762-9343-49c3-8a30-944ba6d2a7eb", - "resource": { - "resourceType": "Observation", - "id": "e3f0a762-9343-49c3-8a30-944ba6d2a7eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "38483-4", - "display": "Creatinine" - } - ], - "text": "Creatinine" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 0.64, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4e78702-bab8-4142-b762-b24634162d40", - "resource": { - "resourceType": "Observation", - "id": "c4e78702-bab8-4142-b762-b24634162d40", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "49765-1", - "display": "Calcium" - } - ], - "text": "Calcium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 9.02, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6abfd13a-c86a-4344-bd60-d8bf64a29c03", - "resource": { - "resourceType": "Observation", - "id": "6abfd13a-c86a-4344-bd60-d8bf64a29c03", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2947-0", - "display": "Sodium" - } - ], - "text": "Sodium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 143.87, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:255ca62e-446c-4fc2-bb77-5d1f25d61f8e", - "resource": { - "resourceType": "Observation", - "id": "255ca62e-446c-4fc2-bb77-5d1f25d61f8e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6298-4", - "display": "Potassium" - } - ], - "text": "Potassium" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 5.09, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:92a1d8c2-3825-4ada-8c64-26d997fed777", - "resource": { - "resourceType": "Observation", - "id": "92a1d8c2-3825-4ada-8c64-26d997fed777", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2069-3", - "display": "Chloride" - } - ], - "text": "Chloride" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 103.9, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6dd8001b-73af-4f87-aab2-6a950cf309a1", - "resource": { - "resourceType": "Observation", - "id": "6dd8001b-73af-4f87-aab2-6a950cf309a1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "20565-8", - "display": "Carbon Dioxide" - } - ], - "text": "Carbon Dioxide" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 28.59, - "unit": "mmol/L", - "system": "http://unitsofmeasure.org", - "code": "mmol/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53abb0b6-550a-427c-9bff-082b1b304257", - "resource": { - "resourceType": "Observation", - "id": "53abb0b6-550a-427c-9bff-082b1b304257", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 176.95, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c74af59-8778-4476-9a7a-c0a9dc570f84", - "resource": { - "resourceType": "Observation", - "id": "1c74af59-8778-4476-9a7a-c0a9dc570f84", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 147.09, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e96b7f96-84cc-4072-aa61-945ec6cfc4b3", - "resource": { - "resourceType": "Observation", - "id": "e96b7f96-84cc-4072-aa61-945ec6cfc4b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 77.51, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3180f15c-d615-487a-88be-94c1091844a6", - "resource": { - "resourceType": "Observation", - "id": "3180f15c-d615-487a-88be-94c1091844a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 70.02, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33ddbc69-bbc9-49f8-a735-ca24076f88a1", - "resource": { - "resourceType": "Observation", - "id": "33ddbc69-bbc9-49f8-a735-ca24076f88a1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:09a92eaa-cd19-4188-880e-627b2c49d282", - "resource": { - "resourceType": "Procedure", - "id": "09a92eaa-cd19-4188-880e-627b2c49d282", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "performedPeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:30:51-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:febfdae2-f92f-4249-8413-80b69b156122", - "resource": { - "resourceType": "MedicationRequest", - "id": "febfdae2-f92f-4249-8413-80b69b156122", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "authoredOn": "2019-12-09T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "display": "Dr. Carlyn477 DuBuque211" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d85d8167-4016-41d1-9c31-756b9929c7a3", - "resource": { - "resourceType": "Claim", - "id": "d85d8167-4016-41d1-9c31-756b9929c7a3", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "created": "2019-12-09T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:febfdae2-f92f-4249-8413-80b69b156122" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1f9834ae-3a62-4f5c-b038-7d6d39e64c9e", - "resource": { - "resourceType": "Immunization", - "id": "1f9834ae-3a62-4f5c-b038-7d6d39e64c9e", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "occurrenceDateTime": "2019-12-09T04:15:51-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4e9ba0d5-b3fd-4054-b06f-52e63ed0a15c", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4e9ba0d5-b3fd-4054-b06f-52e63ed0a15c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "51990-0", - "display": "Basic Metabolic Panel" - } - ], - "text": "Basic Metabolic Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:9e6b720b-bb6a-4dcf-a878-7c5c72b5de85", - "display": "Glucose" - }, - { - "reference": "urn:uuid:765ddade-24ad-4b8b-a2da-7d3ab8dd26ed", - "display": "Urea Nitrogen" - }, - { - "reference": "urn:uuid:e3f0a762-9343-49c3-8a30-944ba6d2a7eb", - "display": "Creatinine" - }, - { - "reference": "urn:uuid:c4e78702-bab8-4142-b762-b24634162d40", - "display": "Calcium" - }, - { - "reference": "urn:uuid:6abfd13a-c86a-4344-bd60-d8bf64a29c03", - "display": "Sodium" - }, - { - "reference": "urn:uuid:255ca62e-446c-4fc2-bb77-5d1f25d61f8e", - "display": "Potassium" - }, - { - "reference": "urn:uuid:92a1d8c2-3825-4ada-8c64-26d997fed777", - "display": "Chloride" - }, - { - "reference": "urn:uuid:6dd8001b-73af-4f87-aab2-6a950cf309a1", - "display": "Carbon Dioxide" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:682ae742-ec78-4b23-a251-bb02dce26c63", - "resource": { - "resourceType": "DiagnosticReport", - "id": "682ae742-ec78-4b23-a251-bb02dce26c63", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "result": [ - { - "reference": "urn:uuid:53abb0b6-550a-427c-9bff-082b1b304257", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:1c74af59-8778-4476-9a7a-c0a9dc570f84", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:e96b7f96-84cc-4072-aa61-945ec6cfc4b3", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:3180f15c-d615-487a-88be-94c1091844a6", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f907c0ed-8084-423e-941f-1983dbc20373", - "resource": { - "resourceType": "Claim", - "id": "f907c0ed-8084-423e-941f-1983dbc20373", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "created": "2019-12-09T04:45:51-08:00", - "provider": { - "reference": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "display": "PCP128586" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1f9834ae-3a62-4f5c-b038-7d6d39e64c9e" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:09a92eaa-cd19-4188-880e-627b2c49d282" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 351.67, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:144a0a57-5508-454e-b497-f1ded7bccd19", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "144a0a57-5508-454e-b497-f1ded7bccd19", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "performer": [ - { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f907c0ed-8084-423e-941f-1983dbc20373" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-09T04:45:51-08:00", - "end": "2020-12-09T04:45:51-08:00" - }, - "created": "2019-12-09T04:45:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:f907c0ed-8084-423e-941f-1983dbc20373" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:02d2ed94-8498-4317-bde6-15685b3922c4" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-09T04:15:51-08:00", - "end": "2019-12-09T04:45:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 351.67, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 70.334, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 281.336, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 351.67, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 351.67, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 393.752, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004", - "resource": { - "resourceType": "Encounter", - "id": "e4c18a30-3495-4590-bc90-c8c213f6f004", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:30:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:30:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:176fbef1-fae8-41dd-9448-77b5c46a154f", - "resource": { - "resourceType": "Claim", - "id": "176fbef1-fae8-41dd-9448-77b5c46a154f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:30:51-08:00" - }, - "created": "2019-12-05T04:30:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c42c40a5-25a0-46dd-a429-51e55bf5c820", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c42c40a5-25a0-46dd-a429-51e55bf5c820", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "176fbef1-fae8-41dd-9448-77b5c46a154f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-05T04:30:51-08:00", - "end": "2020-12-05T04:30:51-08:00" - }, - "created": "2019-12-05T04:30:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:176fbef1-fae8-41dd-9448-77b5c46a154f" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439740005", - "display": "Postoperative follow-up visit (procedure)" - } - ], - "text": "Postoperative follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:30:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e4c18a30-3495-4590-bc90-c8c213f6f004" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2", - "resource": { - "resourceType": "Encounter", - "id": "cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "IMP" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-06T04:39:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-06T04:39:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:be961283-e839-4d04-b38b-deba677062bd", - "resource": { - "resourceType": "Procedure", - "id": "be961283-e839-4d04-b38b-deba677062bd", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" - }, - "performedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:39:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:470b8402-b1df-4cd0-8d7a-03ba12941e61", - "resource": { - "resourceType": "Claim", - "id": "470b8402-b1df-4cd0-8d7a-03ba12941e61", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-06T04:39:51-08:00" - }, - "created": "2019-12-06T04:39:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:be961283-e839-4d04-b38b-deba677062bd" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "encounter": [ - { - "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "net": { - "value": 516.65, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9275a16b-2d24-4c1e-90cc-e7a7085ee9c2", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9275a16b-2d24-4c1e-90cc-e7a7085ee9c2", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "470b8402-b1df-4cd0-8d7a-03ba12941e61" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-06T04:39:51-08:00", - "end": "2020-12-06T04:39:51-08:00" - }, - "created": "2019-12-06T04:39:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:470b8402-b1df-4cd0-8d7a-03ba12941e61" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410410006", - "display": "Screening surveillance (regime/therapy)" - } - ], - "text": "Screening surveillance (regime/therapy)" - }, - "servicedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-06T04:39:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:cbde15f2-3ad3-4fa6-a5ce-381dba62a9f2" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "71651007", - "display": "Mammography (procedure)" - } - ], - "text": "Mammography (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-06T04:39:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 516.65, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 413.32, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608", - "resource": { - "resourceType": "Encounter", - "id": "e7ea7228-a522-422b-92f4-1f1481846608", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Mrs. Robbyn526 DuBuque211" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "individual": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - } - } - ], - "period": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "254837009", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:eb015fef-3373-411e-a4ca-bcf224f1684d", - "resource": { - "resourceType": "Observation", - "id": "eb015fef-3373-411e-a4ca-bcf224f1684d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4548-4", - "display": "Hemoglobin A1c/Hemoglobin.total in Blood" - } - ], - "text": "Hemoglobin A1c/Hemoglobin.total in Blood" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - }, - "effectiveDateTime": "2019-12-09T04:15:51-08:00", - "issued": "2019-12-09T04:15:51.927-08:00", - "valueQuantity": { - "value": 6.34, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff7af662-9815-4168-b804-c2056ba587e2", - "resource": { - "resourceType": "Procedure", - "id": "ff7af662-9815-4168-b804-c2056ba587e2", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - }, - "performedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:36:51-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:89830890-3c8f-4911-8f8d-454837614455", - "display": "Malignant neoplasm of breast (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ed2ee0ba-e4eb-4ae0-9962-0933341559e1", - "resource": { - "resourceType": "MedicationRequest", - "id": "ed2ee0ba-e4eb-4ae0-9962-0933341559e1", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "encounter": { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - }, - "authoredOn": "2019-12-09T04:15:51-08:00", - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "display": "Dr. Heath320 Kautzer186" - }, - "reasonReference": [ - { - "reference": "urn:uuid:023eafd5-a008-4f7e-b974-8e2a25e9e3ad" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0df140b9-02bf-46a3-8022-1270a62d335f", - "resource": { - "resourceType": "Claim", - "id": "0df140b9-02bf-46a3-8022-1270a62d335f", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "created": "2019-12-05T04:51:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:ed2ee0ba-e4eb-4ae0-9962-0933341559e1" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - } - ] - } - ], - "total": { - "value": 263.49, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5d7891b9-ce88-4018-b029-1734949ed66a", - "resource": { - "resourceType": "Claim", - "id": "5d7891b9-ce88-4018-b029-1734949ed66a", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80", - "display": "Robbyn526 DuBuque211" - }, - "billablePeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "created": "2019-12-05T04:51:51-08:00", - "provider": { - "reference": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "display": "CAPE COD HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ff7af662-9815-4168-b804-c2056ba587e2" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "net": { - "value": 13488.18, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3d6162ce-7a2c-4317-b0a4-d568af4b9f90", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3d6162ce-7a2c-4317-b0a4-d568af4b9f90", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "requester": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "performer": [ - { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Medicare" - }, - "beneficiary": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "payor": [ - { - "display": "Medicare" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5d7891b9-ce88-4018-b029-1734949ed66a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:fcf2f472-77ac-47d7-9c9a-a6702bd2bb80" - }, - "billablePeriod": { - "start": "2019-12-05T04:51:51-08:00", - "end": "2020-12-05T04:51:51-08:00" - }, - "created": "2019-12-05T04:51:51-08:00", - "insurer": { - "display": "Medicare" - }, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:5d7891b9-ce88-4018-b029-1734949ed66a" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Medicare" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185389009", - "display": "Follow-up visit (procedure)" - } - ], - "text": "Follow-up visit (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e7ea7228-a522-422b-92f4-1f1481846608" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "312681000", - "display": "Bone density scan (procedure)" - } - ], - "text": "Bone density scan (procedure)" - }, - "servicedPeriod": { - "start": "2019-12-05T04:15:51-08:00", - "end": "2019-12-05T04:51:51-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 13488.18, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2697.6360000000004, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 10790.544000000002, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 13488.18, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 13488.18, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 10790.544000000002, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json deleted file mode 100644 index c4e42e861091..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/Seymour882_Shanahan202_55a5307d-5f23-49c1-9100-7d5c513abca3.json +++ /dev/null @@ -1,16018 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "resource": { - "resourceType": "Patient", - "id": "55a5307d-5f23-49c1-9100-7d5c513abca3", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -792204183018886300 Population seed: 1586368870505
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Lan153 Dicki44" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-birthPlace", - "valueAddress": { - "city": "Cologne", - "state": "North Rhine-Westphalia", - "country": "DE" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.1534940636791636 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 51.84650593632084 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "SS", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-65-3064" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99982974" - }, - { - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0203", - "code": "PPN", - "display": "Passport Number" - } - ], - "text": "Passport Number" - }, - "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", - "value": "X56290648X" - } - ], - "name": [ - { - "use": "official", - "family": "Shanahan202", - "given": [ - "Seymour882" - ], - "prefix": [ - "Mr." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-865-2965", - "use": "home" - } - ], - "gender": "male", - "birthDate": "1967-05-14", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.38437875197168 - }, - { - "url": "longitude", - "valueDecimal": -71.06923432925981 - } - ] - } - ], - "line": [ - "713 Hansen Promenade" - ], - "city": "Everett", - "state": "Massachusetts", - "postalCode": "02149", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-MaritalStatus", - "code": "M", - "display": "M" - } - ], - "text": "M" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "de-DE", - "display": "German (Germany)" - } - ], - "text": "German (Germany)" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "resource": { - "resourceType": "Practitioner", - "id": "eabb178d-5fde-3f2a-b2b0-84b601205578", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999989" - } - ], - "active": true, - "name": [ - { - "family": "Streich926", - "given": [ - "Ja391" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Ja391.Streich926@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede", - "resource": { - "resourceType": "Encounter", - "id": "b85dd21a-4193-44a8-bcc5-f53d16278ede", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-02-28T17:04:38-08:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-02-28T17:04:38-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:21af527f-28ce-41d2-ac17-ba4af06c18a0", - "resource": { - "resourceType": "CareTeam", - "id": "21af527f-28ce-41d2-ac17-ba4af06c18a0", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" - }, - "period": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-04-19T16:32:38-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:267172ad-fda0-4a8f-b5a6-a1725266f55d", - "resource": { - "resourceType": "CarePlan", - "id": "267172ad-fda0-4a8f-b5a6-a1725266f55d", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" - }, - "period": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-04-19T16:32:38-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:21af527f-28ce-41d2-ac17-ba4af06c18a0" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:1f149a81-6393-4202-b62e-6f096f32d053", - "resource": { - "resourceType": "Claim", - "id": "1f149a81-6393-4202-b62e-6f096f32d053", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-02-28T17:04:38-08:00" - }, - "created": "1970-02-28T17:04:38-08:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9fd3fbe8-b7d7-4228-ac55-b4acd544c720", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9fd3fbe8-b7d7-4228-ac55-b4acd544c720", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1f149a81-6393-4202-b62e-6f096f32d053" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "1970-02-28T17:04:38-08:00", - "end": "1971-02-28T17:04:38-08:00" - }, - "created": "1970-02-28T17:04:38-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:1f149a81-6393-4202-b62e-6f096f32d053" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "1970-02-28T16:32:38-08:00", - "end": "1970-02-28T17:04:38-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b85dd21a-4193-44a8-bcc5-f53d16278ede" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488", - "resource": { - "resourceType": "Encounter", - "id": "f7e47867-5416-4c93-a31b-ae8f1f0b0488", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-01T16:47:38-08:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-01T16:47:38-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7290ad08-2670-4c83-a355-bdd794e2ba24", - "resource": { - "resourceType": "CareTeam", - "id": "7290ad08-2670-4c83-a355-bdd794e2ba24", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" - }, - "period": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-08T16:32:38-08:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:c78490fd-321a-4d95-a680-6fcaf0eda0c7", - "resource": { - "resourceType": "CarePlan", - "id": "c78490fd-321a-4d95-a680-6fcaf0eda0c7", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" - }, - "period": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-08T16:32:38-08:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:7290ad08-2670-4c83-a355-bdd794e2ba24" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:826ab08e-cf42-4eab-8c79-9bc4991e9549", - "resource": { - "resourceType": "Claim", - "id": "826ab08e-cf42-4eab-8c79-9bc4991e9549", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-01T16:47:38-08:00" - }, - "created": "2002-02-01T16:47:38-08:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:154d5897-8b4c-467d-aaed-767bd032465d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "154d5897-8b4c-467d-aaed-767bd032465d", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "826ab08e-cf42-4eab-8c79-9bc4991e9549" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2002-02-01T16:47:38-08:00", - "end": "2003-02-01T16:47:38-08:00" - }, - "created": "2002-02-01T16:47:38-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:826ab08e-cf42-4eab-8c79-9bc4991e9549" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2002-02-01T16:32:38-08:00", - "end": "2002-02-01T16:47:38-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f7e47867-5416-4c93-a31b-ae8f1f0b0488" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7", - "resource": { - "resourceType": "Encounter", - "id": "e7a6d528-eb1c-4433-81e0-969a6117f0f7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2002-02-10T16:32:38-08:00", - "end": "2002-02-10T16:47:38-08:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2002-02-10T16:32:38-08:00", - "end": "2002-02-10T16:47:38-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9", - "resource": { - "resourceType": "Condition", - "id": "2fea83b5-42be-43c6-90ff-19b3474404c9", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "active" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" - }, - "onsetDateTime": "2002-02-10T16:32:38-08:00", - "recordedDate": "2002-02-10T16:32:38-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:3ce2c1fb-3037-4243-a36f-615cf940f147", - "resource": { - "resourceType": "Claim", - "id": "3ce2c1fb-3037-4243-a36f-615cf940f147", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2002-02-10T16:32:38-08:00", - "end": "2002-02-10T16:47:38-08:00" - }, - "created": "2002-02-10T16:47:38-08:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7ccdd76f-114e-4f0e-9256-bcf79003d7fd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7ccdd76f-114e-4f0e-9256-bcf79003d7fd", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3ce2c1fb-3037-4243-a36f-615cf940f147" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2002-02-10T16:47:38-08:00", - "end": "2003-02-10T16:47:38-08:00" - }, - "created": "2002-02-10T16:47:38-08:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3ce2c1fb-3037-4243-a36f-615cf940f147" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:2fea83b5-42be-43c6-90ff-19b3474404c9" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "2002-02-10T16:32:38-08:00", - "end": "2002-02-10T16:47:38-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e7a6d528-eb1c-4433-81e0-969a6117f0f7" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162864005", - "display": "Body mass index 30+ - obesity (finding)" - } - ], - "text": "Body mass index 30+ - obesity (finding)" - }, - "servicedPeriod": { - "start": "2002-02-10T16:32:38-08:00", - "end": "2002-02-10T16:47:38-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5", - "resource": { - "resourceType": "Encounter", - "id": "acb018b9-6727-4695-bbd6-cfb1203c8cc5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-04T18:05:38-07:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-04T18:05:38-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2896d320-72d5-41f9-a20b-e0b3167e90fc", - "resource": { - "resourceType": "CareTeam", - "id": "2896d320-72d5-41f9-a20b-e0b3167e90fc", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" - }, - "period": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-11T17:32:38-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:3879053d-fc91-42f2-b08a-dd951e8360d4", - "resource": { - "resourceType": "CarePlan", - "id": "3879053d-fc91-42f2-b08a-dd951e8360d4", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" - }, - "period": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-11T17:32:38-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:2896d320-72d5-41f9-a20b-e0b3167e90fc" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd", - "resource": { - "resourceType": "Claim", - "id": "71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-04T18:05:38-07:00" - }, - "created": "2003-06-04T18:05:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:de0f8980-2c3b-4a18-9c4c-1ad3858eea4f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "de0f8980-2c3b-4a18-9c4c-1ad3858eea4f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2003-06-04T18:05:38-07:00", - "end": "2004-06-04T18:05:38-07:00" - }, - "created": "2003-06-04T18:05:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:71bfefbf-b1aa-4d45-b862-d5f4b9ba8efd" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2003-06-04T17:32:38-07:00", - "end": "2003-06-04T18:05:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:acb018b9-6727-4695-bbd6-cfb1203c8cc5" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551", - "resource": { - "resourceType": "Encounter", - "id": "cc4b50b7-f642-4b8a-b085-dca7290ab551", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2004-11-03T16:52:38-08:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2004-11-03T16:52:38-08:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3c00a98c-9be8-4717-a0b8-ce8c1817a50d", - "resource": { - "resourceType": "CareTeam", - "id": "3c00a98c-9be8-4717-a0b8-ce8c1817a50d", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" - }, - "period": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2007-05-20T17:32:38-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:698109a7-4550-4c9c-ba7a-5d45ca19bb0c", - "resource": { - "resourceType": "CarePlan", - "id": "698109a7-4550-4c9c-ba7a-5d45ca19bb0c", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" - }, - "period": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2007-05-20T17:32:38-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:3c00a98c-9be8-4717-a0b8-ce8c1817a50d" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:43d81a08-a67e-408e-9d9e-d7ff5f3318b0", - "resource": { - "resourceType": "Claim", - "id": "43d81a08-a67e-408e-9d9e-d7ff5f3318b0", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2004-11-03T16:52:38-08:00" - }, - "created": "2004-11-03T16:52:38-08:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" - } - ] - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:99b0cd78-dfd5-4559-a1da-4451f66a01c9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "99b0cd78-dfd5-4559-a1da-4451f66a01c9", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "43d81a08-a67e-408e-9d9e-d7ff5f3318b0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2004-11-03T16:52:38-08:00", - "end": "2005-11-03T16:52:38-08:00" - }, - "created": "2004-11-03T16:52:38-08:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:43d81a08-a67e-408e-9d9e-d7ff5f3318b0" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2004-11-03T16:32:38-08:00", - "end": "2004-11-03T16:52:38-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:cc4b50b7-f642-4b8a-b085-dca7290ab551" - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296", - "resource": { - "resourceType": "Encounter", - "id": "6246c367-6cba-4eab-aae4-3ffe89959296", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", - "resource": { - "resourceType": "Condition", - "id": "dcc3597f-478f-48f3-ac83-cc9767665562", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - }, - "onsetDateTime": "2011-03-31T17:32:38-07:00", - "abatementDateTime": "2011-04-07T17:32:38-07:00", - "recordedDate": "2011-03-31T17:32:38-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:df0dda35-5a8b-452e-8d64-e979f0fae675", - "resource": { - "resourceType": "Procedure", - "id": "df0dda35-5a8b-452e-8d64-e979f0fae675", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - }, - "performedPeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:35:38-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", - "display": "Acute bronchitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7a0bc7a2-a43d-4148-b932-979f71c28b14", - "resource": { - "resourceType": "MedicationRequest", - "id": "7a0bc7a2-a43d-4148-b932-979f71c28b14", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - }, - "authoredOn": "2011-03-31T17:32:38-07:00", - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - }, - "reasonReference": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:cfe5b3d3-3d7c-4e10-a1c8-56375e181d7d", - "resource": { - "resourceType": "Claim", - "id": "cfe5b3d3-3d7c-4e10-a1c8-56375e181d7d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "created": "2011-03-31T17:50:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:7a0bc7a2-a43d-4148-b932-979f71c28b14" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - } - ] - } - ], - "total": { - "value": 6.6, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:83ce868e-2db5-4d10-9f0a-b2f0fd2da042", - "resource": { - "resourceType": "CareTeam", - "id": "83ce868e-2db5-4d10-9f0a-b2f0fd2da042", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - }, - "period": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-05-29T17:32:38-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:e32d95d5-cb65-48a5-9986-cbc861d4c13a", - "resource": { - "resourceType": "CarePlan", - "id": "e32d95d5-cb65-48a5-9986-cbc861d4c13a", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - }, - "period": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-05-29T17:32:38-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:83ce868e-2db5-4d10-9f0a-b2f0fd2da042" - } - ], - "addresses": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:e6c4c2a7-9cb2-45c1-9f16-85a831ff4456", - "resource": { - "resourceType": "Claim", - "id": "e6c4c2a7-9cb2-45c1-9f16-85a831ff4456", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "created": "2011-03-31T17:50:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:df0dda35-5a8b-452e-8d64-e979f0fae675" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "net": { - "value": 9852.22, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3046af09-6075-4fcc-83dd-a4719ba5aa02", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3046af09-6075-4fcc-83dd-a4719ba5aa02", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e6c4c2a7-9cb2-45c1-9f16-85a831ff4456" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-03-31T17:50:38-07:00", - "end": "2012-03-31T17:50:38-07:00" - }, - "created": "2011-03-31T17:50:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:e6c4c2a7-9cb2-45c1-9f16-85a831ff4456" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:6246c367-6cba-4eab-aae4-3ffe89959296" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "servicedPeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "servicedPeriod": { - "start": "2011-03-31T17:32:38-07:00", - "end": "2011-03-31T17:50:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 9852.22, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1970.444, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 7881.776, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 9852.22, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 9852.22, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 7881.776, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "resource": { - "resourceType": "Organization", - "id": "3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "3265f387-6c51-32ee-8f6d-b2a89caa34d5" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN", - "telecom": [ - { - "system": "phone", - "value": "617-884-8300" - } - ], - "address": [ - { - "line": [ - "151 EVERETT AVENUE" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "2150", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", - "resource": { - "resourceType": "Practitioner", - "id": "fe82a096-86b9-344b-b405-bd24b6ad4715", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906709" - } - ], - "active": true, - "name": [ - { - "family": "Hoppe518", - "given": [ - "Devin82" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Devin82.Hoppe518@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "151 EVERETT AVENUE" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "2150", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb", - "resource": { - "resourceType": "Encounter", - "id": "ce658d96-f9dd-420c-86b7-822b01e259cb", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-04-10T17:32:38-07:00", - "end": "2011-04-10T17:47:38-07:00" - }, - "individual": { - "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", - "display": "Dr. Devin82 Hoppe518" - } - } - ], - "period": { - "start": "2011-04-10T17:32:38-07:00", - "end": "2011-04-10T17:47:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "display": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e8f655cf-40f6-4d92-8370-03bb6961b99c", - "resource": { - "resourceType": "Immunization", - "id": "e8f655cf-40f6-4d92-8370-03bb6961b99c", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" - }, - "occurrenceDateTime": "2011-04-10T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:197f75bd-6b12-4317-8c68-f05c9deced7b", - "resource": { - "resourceType": "Claim", - "id": "197f75bd-6b12-4317-8c68-f05c9deced7b", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2011-04-10T17:32:38-07:00", - "end": "2011-04-10T17:47:38-07:00" - }, - "created": "2011-04-10T17:47:38-07:00", - "provider": { - "reference": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "display": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e8f655cf-40f6-4d92-8370-03bb6961b99c" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a71232d3-d98a-4fba-88ef-ad4be18d4142", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a71232d3-d98a-4fba-88ef-ad4be18d4142", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" - }, - "performer": [ - { - "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "197f75bd-6b12-4317-8c68-f05c9deced7b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-04-10T17:47:38-07:00", - "end": "2012-04-10T17:47:38-07:00" - }, - "created": "2011-04-10T17:47:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:197f75bd-6b12-4317-8c68-f05c9deced7b" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "702927004", - "display": "Urgent care clinic (procedure)" - } - ], - "text": "Urgent care clinic (procedure)" - }, - "servicedPeriod": { - "start": "2011-04-10T17:32:38-07:00", - "end": "2011-04-10T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ce658d96-f9dd-420c-86b7-822b01e259cb" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2011-04-10T17:32:38-07:00", - "end": "2011-04-10T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "resource": { - "resourceType": "Organization", - "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAMBRIDGE HEALTH ALLIANCE", - "telecom": [ - { - "system": "phone", - "value": "6176652300" - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "resource": { - "resourceType": "Practitioner", - "id": "e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999959" - } - ], - "active": true, - "name": [ - { - "family": "Harvey63", - "given": [ - "Alla648" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Alla648.Harvey63@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca", - "resource": { - "resourceType": "Encounter", - "id": "85983bde-e67b-407e-97e9-a635fd9accca", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "individual": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "display": "Dr. Alla648 Harvey63" - } - } - ], - "period": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "display": "CAMBRIDGE HEALTH ALLIANCE" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e", - "resource": { - "resourceType": "Condition", - "id": "1e0f6578-019f-4592-a666-a2438a39fd0e", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "39848009", - "display": "Whiplash injury to neck" - } - ], - "text": "Whiplash injury to neck" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - }, - "onsetDateTime": "2011-05-12T17:32:38-07:00", - "abatementDateTime": "2011-06-02T17:32:38-07:00", - "recordedDate": "2011-05-12T17:32:38-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:e570df1d-5083-4b02-b418-b871f10956a5", - "resource": { - "resourceType": "MedicationRequest", - "id": "e570df1d-5083-4b02-b418-b871f10956a5", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - }, - "authoredOn": "2011-05-12T17:32:38-07:00", - "requester": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "display": "Dr. Alla648 Harvey63" - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:28e0d2b8-4c92-482d-862c-fce1dc363f92", - "resource": { - "resourceType": "Claim", - "id": "28e0d2b8-4c92-482d-862c-fce1dc363f92", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "created": "2011-05-12T18:32:38-07:00", - "provider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "display": "CAMBRIDGE HEALTH ALLIANCE" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:e570df1d-5083-4b02-b418-b871f10956a5" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - } - ] - } - ], - "total": { - "value": 7.92, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b", - "resource": { - "resourceType": "CareTeam", - "id": "7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b", - "status": "inactive", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - }, - "period": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-06-02T17:32:38-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "display": "Dr. Alla648 Harvey63" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "display": "CAMBRIDGE HEALTH ALLIANCE" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "39848009", - "display": "Whiplash injury to neck" - } - ], - "text": "Whiplash injury to neck" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "display": "CAMBRIDGE HEALTH ALLIANCE" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:629069fe-9cfa-42d4-8f96-b6e1b5ef8118", - "resource": { - "resourceType": "CarePlan", - "id": "629069fe-9cfa-42d4-8f96-b6e1b5ef8118", - "text": { - "status": "generated", - "div": "
    Care Plan for Musculoskeletal care.
    Activities:
    • Musculoskeletal care
    • Musculoskeletal care

    Care plan is meant to treat Whiplash injury to neck.
    " - }, - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "408869004", - "display": "Musculoskeletal care" - } - ], - "text": "Musculoskeletal care" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - }, - "period": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-06-02T17:32:38-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:7c0a6c52-d908-4ccb-9c4b-fc9ffe357c2b" - } - ], - "addresses": [ - { - "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266694003", - "display": "Heat therapy" - } - ], - "text": "Heat therapy" - }, - "status": "completed", - "location": { - "display": "CAMBRIDGE HEALTH ALLIANCE" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "183051005", - "display": "Recommendation to rest" - } - ], - "text": "Recommendation to rest" - }, - "status": "completed", - "location": { - "display": "CAMBRIDGE HEALTH ALLIANCE" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:9ed2138b-f8e2-4c9c-ad68-18f487d5d20d", - "resource": { - "resourceType": "Claim", - "id": "9ed2138b-f8e2-4c9c-ad68-18f487d5d20d", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "created": "2011-05-12T18:32:38-07:00", - "provider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "display": "CAMBRIDGE HEALTH ALLIANCE" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "39848009", - "display": "Whiplash injury to neck" - } - ], - "text": "Whiplash injury to neck" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3a2d8bdb-4406-4172-b753-2d419d6f2bb4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3a2d8bdb-4406-4172-b753-2d419d6f2bb4", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" - }, - "performer": [ - { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9ed2138b-f8e2-4c9c-ad68-18f487d5d20d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-05-12T18:32:38-07:00", - "end": "2012-05-12T18:32:38-07:00" - }, - "created": "2011-05-12T18:32:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:9ed2138b-f8e2-4c9c-ad68-18f487d5d20d" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:1e0f6578-019f-4592-a666-a2438a39fd0e" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - }, - "servicedPeriod": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:85983bde-e67b-407e-97e9-a635fd9accca" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "39848009", - "display": "Whiplash injury to neck" - } - ], - "text": "Whiplash injury to neck" - }, - "servicedPeriod": { - "start": "2011-05-12T17:32:38-07:00", - "end": "2011-05-12T18:32:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "20", - "display": "Urgent Care Facility" - } - ] - } - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 0.0, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "resource": { - "resourceType": "Organization", - "id": "7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "7f7c08c8-1c39-3b26-bd34-5122a003dd91" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP83180", - "telecom": [ - { - "system": "phone", - "value": "617-389-6951" - } - ], - "address": [ - { - "line": [ - "107 FERRY ST" - ], - "city": "EVERETT", - "state": "MA", - "postalCode": "02149-4940", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "resource": { - "resourceType": "Practitioner", - "id": "12810b32-78af-32f0-b85c-8177e3f2e2e2", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999959759" - } - ], - "active": true, - "name": [ - { - "family": "Goldner995", - "given": [ - "Doretta917" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Doretta917.Goldner995@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "107 FERRY ST" - ], - "city": "EVERETT", - "state": "MA", - "postalCode": "02149-4940", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839", - "resource": { - "resourceType": "Encounter", - "id": "168913b6-d1f2-4f0b-a74f-7ac898c14839", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2011-05-29T17:32:38-07:00", - "end": "2011-05-29T17:47:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2011-05-29T17:32:38-07:00", - "end": "2011-05-29T17:47:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6565831f-d203-479b-9376-d77b0ee4234a", - "resource": { - "resourceType": "Observation", - "id": "6565831f-d203-479b-9376-d77b0ee4234a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2af09e1b-6451-43a5-9edc-701e1184fd8d", - "resource": { - "resourceType": "Observation", - "id": "2af09e1b-6451-43a5-9edc-701e1184fd8d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34171b9d-e2fa-4caa-8610-c570df877412", - "resource": { - "resourceType": "Observation", - "id": "34171b9d-e2fa-4caa-8610-c570df877412", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 88.1, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e5215a80-98f0-4644-bb1f-a3f9698e7edb", - "resource": { - "resourceType": "Observation", - "id": "e5215a80-98f0-4644-bb1f-a3f9698e7edb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 30.38, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d09b9013-96f9-4d2b-af3b-00e058f11f92", - "resource": { - "resourceType": "Observation", - "id": "d09b9013-96f9-4d2b-af3b-00e058f11f92", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 107, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7dddbe4-8f34-4938-808a-571ca75b43c6", - "resource": { - "resourceType": "Observation", - "id": "b7dddbe4-8f34-4938-808a-571ca75b43c6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 72, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d3ea5a9d-6a55-4b5c-8d47-0fc57ae0e09a", - "resource": { - "resourceType": "Observation", - "id": "d3ea5a9d-6a55-4b5c-8d47-0fc57ae0e09a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3e50573-0d80-4b82-8068-bccc745935bf", - "resource": { - "resourceType": "Observation", - "id": "e3e50573-0d80-4b82-8068-bccc745935bf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "effectiveDateTime": "2011-05-29T17:32:38-07:00", - "issued": "2011-05-29T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:edb585d8-20f2-441f-a9a7-655ca5cb7b9a", - "resource": { - "resourceType": "Immunization", - "id": "edb585d8-20f2-441f-a9a7-655ca5cb7b9a", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - }, - "occurrenceDateTime": "2011-05-29T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:3f213d14-94fa-41b7-9705-9b02524cc235", - "resource": { - "resourceType": "Claim", - "id": "3f213d14-94fa-41b7-9705-9b02524cc235", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2011-05-29T17:32:38-07:00", - "end": "2011-05-29T17:47:38-07:00" - }, - "created": "2011-05-29T17:47:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:edb585d8-20f2-441f-a9a7-655ca5cb7b9a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ca7edb6e-e0ce-4ff0-8702-f38e789fb7c9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ca7edb6e-e0ce-4ff0-8702-f38e789fb7c9", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3f213d14-94fa-41b7-9705-9b02524cc235" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2011-05-29T17:47:38-07:00", - "end": "2012-05-29T17:47:38-07:00" - }, - "created": "2011-05-29T17:47:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:3f213d14-94fa-41b7-9705-9b02524cc235" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2011-05-29T17:32:38-07:00", - "end": "2011-05-29T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:168913b6-d1f2-4f0b-a74f-7ac898c14839" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2011-05-29T17:32:38-07:00", - "end": "2011-05-29T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b", - "resource": { - "resourceType": "Encounter", - "id": "0ed37b12-189d-4f77-a476-322d8b02b50b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8fb30fb0-d2fe-463d-a117-d6d9722471a3", - "resource": { - "resourceType": "Observation", - "id": "8fb30fb0-d2fe-463d-a117-d6d9722471a3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d58ef249-de16-4e21-a868-4cee6108f037", - "resource": { - "resourceType": "Observation", - "id": "d58ef249-de16-4e21-a868-4cee6108f037", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed82af46-5e99-49c2-8d97-18126376d387", - "resource": { - "resourceType": "Observation", - "id": "ed82af46-5e99-49c2-8d97-18126376d387", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 82.8, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc96cec5-a405-43ba-90ef-456c4f8dfa7a", - "resource": { - "resourceType": "Observation", - "id": "fc96cec5-a405-43ba-90ef-456c4f8dfa7a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 28.56, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71eb7072-6c18-4e3c-a350-6fe28fc2eb0f", - "resource": { - "resourceType": "Observation", - "id": "71eb7072-6c18-4e3c-a350-6fe28fc2eb0f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba73a834-1e2a-4c2d-bc61-86dceb0144d3", - "resource": { - "resourceType": "Observation", - "id": "ba73a834-1e2a-4c2d-bc61-86dceb0144d3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8093bc65-4c7a-4d08-bc8c-678045750832", - "resource": { - "resourceType": "Observation", - "id": "8093bc65-4c7a-4d08-bc8c-678045750832", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5a807eb-18d7-4732-bc99-603aa0bbb318", - "resource": { - "resourceType": "Observation", - "id": "a5a807eb-18d7-4732-bc99-603aa0bbb318", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 187.47, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d58bc55-be80-4a32-af90-d989672090ec", - "resource": { - "resourceType": "Observation", - "id": "5d58bc55-be80-4a32-af90-d989672090ec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 118.26, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c1516684-18fb-46e5-b22c-8cb5d8abcb3d", - "resource": { - "resourceType": "Observation", - "id": "c1516684-18fb-46e5-b22c-8cb5d8abcb3d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 102.89, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", - "resource": { - "resourceType": "Observation", - "id": "d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 60.93, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ae060af-1729-4b94-983b-8c4b8a9ec7dc", - "resource": { - "resourceType": "Observation", - "id": "2ae060af-1729-4b94-983b-8c4b8a9ec7dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 7.5831, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c43ca49-92da-48ea-9cf5-9ed8fecfc560", - "resource": { - "resourceType": "Observation", - "id": "5c43ca49-92da-48ea-9cf5-9ed8fecfc560", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 4.9759, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:00676671-b269-48dd-9bc8-c62061d3f270", - "resource": { - "resourceType": "Observation", - "id": "00676671-b269-48dd-9bc8-c62061d3f270", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 13.888, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6511d2e-eaba-4215-af31-3ac7fe071181", - "resource": { - "resourceType": "Observation", - "id": "e6511d2e-eaba-4215-af31-3ac7fe071181", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 45.689, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9359b02-c057-4b87-87a5-511feac9b3f5", - "resource": { - "resourceType": "Observation", - "id": "c9359b02-c057-4b87-87a5-511feac9b3f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 90.832, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9039dc90-96d1-4110-a207-163897880da6", - "resource": { - "resourceType": "Observation", - "id": "9039dc90-96d1-4110-a207-163897880da6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 28.772, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7344abd2-7d08-4043-801d-766dbfa2f32d", - "resource": { - "resourceType": "Observation", - "id": "7344abd2-7d08-4043-801d-766dbfa2f32d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 33.495, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eda3d747-7b0b-4c44-b10d-32929eb24e20", - "resource": { - "resourceType": "Observation", - "id": "eda3d747-7b0b-4c44-b10d-32929eb24e20", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 39.944, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3523c655-0212-44a7-9517-0d2a318a5d78", - "resource": { - "resourceType": "Observation", - "id": "3523c655-0212-44a7-9517-0d2a318a5d78", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 345.47, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:941bd060-2882-4cc9-9e13-d29b9822fe4a", - "resource": { - "resourceType": "Observation", - "id": "941bd060-2882-4cc9-9e13-d29b9822fe4a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 317.47, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e7f57e9-d3a4-4604-8c13-f414e224ccdb", - "resource": { - "resourceType": "Observation", - "id": "5e7f57e9-d3a4-4604-8c13-f414e224ccdb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueQuantity": { - "value": 9.6379, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69f13519-8224-4db2-91e4-a625363eac58", - "resource": { - "resourceType": "Observation", - "id": "69f13519-8224-4db2-91e4-a625363eac58", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36b6a81b-eaea-4d57-9f0b-29f96a3e9861", - "resource": { - "resourceType": "Procedure", - "id": "36b6a81b-eaea-4d57-9f0b-29f96a3e9861", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "performedPeriod": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T17:47:38-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5f878a58-a480-4834-9dec-34cb4ede34e6", - "resource": { - "resourceType": "Immunization", - "id": "5f878a58-a480-4834-9dec-34cb4ede34e6", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "occurrenceDateTime": "2013-06-02T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:54d59b25-230c-4666-b1c9-7ffc76c5eaac", - "resource": { - "resourceType": "DiagnosticReport", - "id": "54d59b25-230c-4666-b1c9-7ffc76c5eaac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "result": [ - { - "reference": "urn:uuid:a5a807eb-18d7-4732-bc99-603aa0bbb318", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:5d58bc55-be80-4a32-af90-d989672090ec", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:c1516684-18fb-46e5-b22c-8cb5d8abcb3d", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:d2bb7f5d-4db0-4737-beb9-31ee0ef99e9c", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:1a8aa626-bf14-4a52-aae3-a1d4a3338976", - "resource": { - "resourceType": "DiagnosticReport", - "id": "1a8aa626-bf14-4a52-aae3-a1d4a3338976", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - }, - "effectiveDateTime": "2013-06-02T17:32:38-07:00", - "issued": "2013-06-02T17:32:38.789-07:00", - "result": [ - { - "reference": "urn:uuid:2ae060af-1729-4b94-983b-8c4b8a9ec7dc", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:5c43ca49-92da-48ea-9cf5-9ed8fecfc560", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:00676671-b269-48dd-9bc8-c62061d3f270", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:e6511d2e-eaba-4215-af31-3ac7fe071181", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:c9359b02-c057-4b87-87a5-511feac9b3f5", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:9039dc90-96d1-4110-a207-163897880da6", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:7344abd2-7d08-4043-801d-766dbfa2f32d", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:eda3d747-7b0b-4c44-b10d-32929eb24e20", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:3523c655-0212-44a7-9517-0d2a318a5d78", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:941bd060-2882-4cc9-9e13-d29b9822fe4a", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:5e7f57e9-d3a4-4604-8c13-f414e224ccdb", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:dc89f63d-273b-4fab-bcab-28d24d55e146", - "resource": { - "resourceType": "Claim", - "id": "dc89f63d-273b-4fab-bcab-28d24d55e146", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "created": "2013-06-02T18:02:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:5f878a58-a480-4834-9dec-34cb4ede34e6" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:36b6a81b-eaea-4d57-9f0b-29f96a3e9861" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 423.86, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5f261924-bfa9-45a7-a125-4dadd8ed66ab", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5f261924-bfa9-45a7-a125-4dadd8ed66ab", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "dc89f63d-273b-4fab-bcab-28d24d55e146" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2013-06-02T18:02:38-07:00", - "end": "2014-06-02T18:02:38-07:00" - }, - "created": "2013-06-02T18:02:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:dc89f63d-273b-4fab-bcab-28d24d55e146" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0ed37b12-189d-4f77-a476-322d8b02b50b" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2013-06-02T17:32:38-07:00", - "end": "2013-06-02T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 423.86, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 84.772, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 339.088, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 423.86, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 423.86, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 451.504, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0", - "resource": { - "resourceType": "Encounter", - "id": "7ffed3ba-6e1a-4502-8b6b-163941059ba0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:54c48d97-ee77-4b4f-b7b0-d48e9947ca68", - "resource": { - "resourceType": "Observation", - "id": "54c48d97-ee77-4b4f-b7b0-d48e9947ca68", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4a36425d-9896-4218-90df-269ae990c1ee", - "resource": { - "resourceType": "Observation", - "id": "4a36425d-9896-4218-90df-269ae990c1ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b7fb02b-d701-41e2-90fe-460b353773d8", - "resource": { - "resourceType": "Observation", - "id": "3b7fb02b-d701-41e2-90fe-460b353773d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 84.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e39ac64-944f-4ebf-8a8b-6ceb9811052d", - "resource": { - "resourceType": "Observation", - "id": "0e39ac64-944f-4ebf-8a8b-6ceb9811052d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 29.15, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cc1bec31-bfb0-4acc-9601-9a41000e07fd", - "resource": { - "resourceType": "Observation", - "id": "cc1bec31-bfb0-4acc-9601-9a41000e07fd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b5e8389a-f729-47a7-99ae-a1b2f91154e3", - "resource": { - "resourceType": "Observation", - "id": "b5e8389a-f729-47a7-99ae-a1b2f91154e3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0afd041a-ca38-44ca-83f9-3a5f4f0c9a12", - "resource": { - "resourceType": "Observation", - "id": "0afd041a-ca38-44ca-83f9-3a5f4f0c9a12", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:beb3b42d-35a0-4c8e-ac9c-8e4a92950aa1", - "resource": { - "resourceType": "Observation", - "id": "beb3b42d-35a0-4c8e-ac9c-8e4a92950aa1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "effectiveDateTime": "2015-06-07T17:32:38-07:00", - "issued": "2015-06-07T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e560761e-0142-4e6a-bd11-99cb7313570f", - "resource": { - "resourceType": "Procedure", - "id": "e560761e-0142-4e6a-bd11-99cb7313570f", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "performedPeriod": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T17:47:38-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:196ac3fc-72d4-43cf-9e23-7bdd6a09bdff", - "resource": { - "resourceType": "Immunization", - "id": "196ac3fc-72d4-43cf-9e23-7bdd6a09bdff", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - }, - "occurrenceDateTime": "2015-06-07T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8b2e8d05-124d-43f0-a35e-0a2a086c50ce", - "resource": { - "resourceType": "Claim", - "id": "8b2e8d05-124d-43f0-a35e-0a2a086c50ce", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "created": "2015-06-07T18:02:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:196ac3fc-72d4-43cf-9e23-7bdd6a09bdff" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e560761e-0142-4e6a-bd11-99cb7313570f" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 451.20, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1952734b-fe29-4594-9861-2fc402104c64", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1952734b-fe29-4594-9861-2fc402104c64", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8b2e8d05-124d-43f0-a35e-0a2a086c50ce" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2015-06-07T18:02:38-07:00", - "end": "2016-06-07T18:02:38-07:00" - }, - "created": "2015-06-07T18:02:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:8b2e8d05-124d-43f0-a35e-0a2a086c50ce" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7ffed3ba-6e1a-4502-8b6b-163941059ba0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2015-06-07T17:32:38-07:00", - "end": "2015-06-07T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 451.20, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 90.24000000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 360.96000000000004, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 451.20, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 451.20, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 473.37600000000003, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25", - "resource": { - "resourceType": "Encounter", - "id": "f776bcfc-3dbf-498e-abce-2ae011870b25", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:feadcde7-7aec-43fa-b41e-791efb5cbb71", - "resource": { - "resourceType": "Observation", - "id": "feadcde7-7aec-43fa-b41e-791efb5cbb71", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a6763509-d36a-4856-8d13-f92485daa2b4", - "resource": { - "resourceType": "Observation", - "id": "a6763509-d36a-4856-8d13-f92485daa2b4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bfeb41b7-cc6e-41b6-81f5-02ddb0445f09", - "resource": { - "resourceType": "Observation", - "id": "bfeb41b7-cc6e-41b6-81f5-02ddb0445f09", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 87.7, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:483bc0eb-821d-444c-be38-90f064ad3cd2", - "resource": { - "resourceType": "Observation", - "id": "483bc0eb-821d-444c-be38-90f064ad3cd2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 30.24, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b74cd60d-20aa-44b5-9e11-bc581a7c6ed8", - "resource": { - "resourceType": "Observation", - "id": "b74cd60d-20aa-44b5-9e11-bc581a7c6ed8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 132, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84fc8272-3077-4a79-aaae-726283414f89", - "resource": { - "resourceType": "Observation", - "id": "84fc8272-3077-4a79-aaae-726283414f89", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 90, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a2fc9e1-5bec-44e5-a4cf-b9c057b4ad7c", - "resource": { - "resourceType": "Observation", - "id": "3a2fc9e1-5bec-44e5-a4cf-b9c057b4ad7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a9f8b8cb-9156-4017-a41e-4da710ffc426", - "resource": { - "resourceType": "Observation", - "id": "a9f8b8cb-9156-4017-a41e-4da710ffc426", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2093-3", - "display": "Total Cholesterol" - } - ], - "text": "Total Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 160.74, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b9229d8e-f291-4116-8b78-8c53005413f3", - "resource": { - "resourceType": "Observation", - "id": "b9229d8e-f291-4116-8b78-8c53005413f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2571-8", - "display": "Triglycerides" - } - ], - "text": "Triglycerides" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 108.59, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f4e16d54-96c2-4629-ac36-31ace4f6f3a9", - "resource": { - "resourceType": "Observation", - "id": "f4e16d54-96c2-4629-ac36-31ace4f6f3a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "18262-6", - "display": "Low Density Lipoprotein Cholesterol" - } - ], - "text": "Low Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 64.9, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d6b5d8d8-e511-430f-93e7-2d48550b69d8", - "resource": { - "resourceType": "Observation", - "id": "d6b5d8d8-e511-430f-93e7-2d48550b69d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "2085-9", - "display": "High Density Lipoprotein Cholesterol" - } - ], - "text": "High Density Lipoprotein Cholesterol" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueQuantity": { - "value": 74.12, - "unit": "mg/dL", - "system": "http://unitsofmeasure.org", - "code": "mg/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c2698ad-ac28-486b-a4d3-3a1b358735fb", - "resource": { - "resourceType": "Observation", - "id": "7c2698ad-ac28-486b-a4d3-3a1b358735fb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb6a390b-1c12-4d61-bba2-50dea47dfd0f", - "resource": { - "resourceType": "Immunization", - "id": "cb6a390b-1c12-4d61-bba2-50dea47dfd0f", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "occurrenceDateTime": "2017-05-14T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c8b9efc2-90bb-43ad-9836-04a5c6a10735", - "resource": { - "resourceType": "Immunization", - "id": "c8b9efc2-90bb-43ad-9836-04a5c6a10735", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "occurrenceDateTime": "2017-05-14T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1c25e149-5d29-4c64-9d88-f67a78d6a0fb", - "resource": { - "resourceType": "DiagnosticReport", - "id": "1c25e149-5d29-4c64-9d88-f67a78d6a0fb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "57698-3", - "display": "Lipid Panel" - } - ], - "text": "Lipid Panel" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - }, - "effectiveDateTime": "2017-05-14T17:32:38-07:00", - "issued": "2017-05-14T17:32:38.789-07:00", - "result": [ - { - "reference": "urn:uuid:a9f8b8cb-9156-4017-a41e-4da710ffc426", - "display": "Total Cholesterol" - }, - { - "reference": "urn:uuid:b9229d8e-f291-4116-8b78-8c53005413f3", - "display": "Triglycerides" - }, - { - "reference": "urn:uuid:f4e16d54-96c2-4629-ac36-31ace4f6f3a9", - "display": "Low Density Lipoprotein Cholesterol" - }, - { - "reference": "urn:uuid:d6b5d8d8-e511-430f-93e7-2d48550b69d8", - "display": "High Density Lipoprotein Cholesterol" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:076e6cbb-8e89-40b0-87a1-95f307b3bd7e", - "resource": { - "resourceType": "Claim", - "id": "076e6cbb-8e89-40b0-87a1-95f307b3bd7e", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "created": "2017-05-14T17:47:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:cb6a390b-1c12-4d61-bba2-50dea47dfd0f" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c8b9efc2-90bb-43ad-9836-04a5c6a10735" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cc4a346a-b5e5-49cd-a056-f600d538a65f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cc4a346a-b5e5-49cd-a056-f600d538a65f", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "076e6cbb-8e89-40b0-87a1-95f307b3bd7e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2017-05-14T17:47:38-07:00", - "end": "2018-05-14T17:47:38-07:00" - }, - "created": "2017-05-14T17:47:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:076e6cbb-8e89-40b0-87a1-95f307b3bd7e" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f776bcfc-3dbf-498e-abce-2ae011870b25" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 224.83200000000002, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98", - "resource": { - "resourceType": "Encounter", - "id": "bbeaf98d-63d5-4ade-be01-1f622ead5a98", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:23:38-07:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:23:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:223ecd8e-0fb0-4a36-af9d-30209d7e8087", - "resource": { - "resourceType": "Procedure", - "id": "223ecd8e-0fb0-4a36-af9d-30209d7e8087", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" - }, - "performedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:08:38-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ebde62e3-4086-4e07-bbb1-24f532835804", - "resource": { - "resourceType": "Claim", - "id": "ebde62e3-4086-4e07-bbb1-24f532835804", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:23:38-07:00" - }, - "created": "2017-05-14T18:23:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:223ecd8e-0fb0-4a36-af9d-30209d7e8087" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "encounter": [ - { - "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" - } - ] - }, - { - "sequence": 2, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "net": { - "value": 18980.64, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5b200d19-06de-42b6-85ce-eac88e5aaa45", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5b200d19-06de-42b6-85ce-eac88e5aaa45", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ebde62e3-4086-4e07-bbb1-24f532835804" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2017-05-14T18:23:38-07:00", - "end": "2018-05-14T18:23:38-07:00" - }, - "created": "2017-05-14T18:23:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:ebde62e3-4086-4e07-bbb1-24f532835804" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - }, - "servicedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:23:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:bbeaf98d-63d5-4ade-be01-1f622ead5a98" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "73761001", - "display": "Colonoscopy" - } - ], - "text": "Colonoscopy" - }, - "servicedPeriod": { - "start": "2017-05-14T17:32:38-07:00", - "end": "2017-05-14T18:23:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 18980.64, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 3796.128, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 15184.512, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 18980.64, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 18980.64, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 15184.512, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29", - "resource": { - "resourceType": "Encounter", - "id": "c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:48468fdb-4817-42ff-9b70-06794275b9a8", - "resource": { - "resourceType": "Observation", - "id": "48468fdb-4817-42ff-9b70-06794275b9a8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3738c5c3-2754-44a3-8316-39068d145e91", - "resource": { - "resourceType": "Observation", - "id": "3738c5c3-2754-44a3-8316-39068d145e91", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:88483de3-63c0-48e7-b3b4-5a84b4b2d4cb", - "resource": { - "resourceType": "Observation", - "id": "88483de3-63c0-48e7-b3b4-5a84b4b2d4cb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 82.7, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5dc7c11-8e61-4cb6-bf4c-abe89f1743ab", - "resource": { - "resourceType": "Observation", - "id": "a5dc7c11-8e61-4cb6-bf4c-abe89f1743ab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 28.51, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6081858c-f14a-4780-aabd-5797c532fd5a", - "resource": { - "resourceType": "Observation", - "id": "6081858c-f14a-4780-aabd-5797c532fd5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 120, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cba93d27-bf57-4085-86dc-3f9814913545", - "resource": { - "resourceType": "Observation", - "id": "cba93d27-bf57-4085-86dc-3f9814913545", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 66, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9a36f421-3850-4e11-8565-2cc9001bb3dc", - "resource": { - "resourceType": "Observation", - "id": "9a36f421-3850-4e11-8565-2cc9001bb3dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c315d2e8-6337-4a41-9623-f590369795d0", - "resource": { - "resourceType": "Observation", - "id": "c315d2e8-6337-4a41-9623-f590369795d0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "effectiveDateTime": "2018-05-20T17:32:38-07:00", - "issued": "2018-05-20T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:81abebcd-0ab9-4d20-8a5f-c7726af47076", - "resource": { - "resourceType": "Procedure", - "id": "81abebcd-0ab9-4d20-8a5f-c7726af47076", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "performedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T17:47:38-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ad206db7-3a4e-4243-9964-e9a011e7e599", - "resource": { - "resourceType": "Immunization", - "id": "ad206db7-3a4e-4243-9964-e9a011e7e599", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "occurrenceDateTime": "2018-05-20T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:70783cd0-a45e-4d39-abef-cdb681275231", - "resource": { - "resourceType": "Immunization", - "id": "70783cd0-a45e-4d39-abef-cdb681275231", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "occurrenceDateTime": "2018-05-20T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7bb1566f-b6c8-4528-8fb6-fdcdd490ba48", - "resource": { - "resourceType": "Immunization", - "id": "7bb1566f-b6c8-4528-8fb6-fdcdd490ba48", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - }, - "occurrenceDateTime": "2018-05-20T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b173915f-7894-4b30-b6ed-2babd254dc71", - "resource": { - "resourceType": "Claim", - "id": "b173915f-7894-4b30-b6ed-2babd254dc71", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "created": "2018-05-20T18:02:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ad206db7-3a4e-4243-9964-e9a011e7e599" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:70783cd0-a45e-4d39-abef-cdb681275231" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7bb1566f-b6c8-4528-8fb6-fdcdd490ba48" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:81abebcd-0ab9-4d20-8a5f-c7726af47076" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - }, - { - "sequence": 5, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "net": { - "value": 665.14, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f41e5d5d-cc37-42fe-b983-b22cca1855e3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f41e5d5d-cc37-42fe-b983-b22cca1855e3", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b173915f-7894-4b30-b6ed-2babd254dc71" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2018-05-20T18:02:38-07:00", - "end": "2019-05-20T18:02:38-07:00" - }, - "created": "2018-05-20T18:02:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b173915f-7894-4b30-b6ed-2babd254dc71" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c09bb8a2-82b9-4ea4-b67e-b3ced6eabd29" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "121", - "display": "zoster" - } - ], - "text": "zoster" - }, - "servicedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationSequence": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 4, - "informationSequence": [ - 3 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "servicedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "servicedPeriod": { - "start": "2018-05-20T17:32:38-07:00", - "end": "2018-05-20T18:02:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 665.14, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 133.028, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 532.112, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 665.14, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 665.14, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 869.36, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04", - "resource": { - "resourceType": "Encounter", - "id": "f3d3fd85-ab22-43fb-8c38-d25621657a04", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2019-05-26T17:32:38-07:00", - "end": "2019-05-26T17:47:38-07:00" - }, - "individual": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "display": "Dr. Doretta917 Goldner995" - } - } - ], - "period": { - "start": "2019-05-26T17:32:38-07:00", - "end": "2019-05-26T17:47:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cb5a50cc-b8f6-4a5b-b2c1-ae3504d37bea", - "resource": { - "resourceType": "Observation", - "id": "cb5a50cc-b8f6-4a5b-b2c1-ae3504d37bea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:96b0ce29-9b6e-47c0-906c-21993090b3cc", - "resource": { - "resourceType": "Observation", - "id": "96b0ce29-9b6e-47c0-906c-21993090b3cc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8da3621a-3caa-4949-8306-77c6df70f0b8", - "resource": { - "resourceType": "Observation", - "id": "8da3621a-3caa-4949-8306-77c6df70f0b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 81.2, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3fdaed1f-a5ad-490f-8ba9-42f1f97fbbcc", - "resource": { - "resourceType": "Observation", - "id": "3fdaed1f-a5ad-490f-8ba9-42f1f97fbbcc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 28, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fdfd4246-dacb-4452-b200-2e4c43d4bd28", - "resource": { - "resourceType": "Observation", - "id": "fdfd4246-dacb-4452-b200-2e4c43d4bd28", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 72, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 102, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:87f9f1bb-f715-43bc-a731-15e167f0a205", - "resource": { - "resourceType": "Observation", - "id": "87f9f1bb-f715-43bc-a731-15e167f0a205", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34f92871-ad58-40d2-98c5-a54529678d69", - "resource": { - "resourceType": "Observation", - "id": "34f92871-ad58-40d2-98c5-a54529678d69", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a95619d-84a9-41ea-aca9-ec18f5b74f73", - "resource": { - "resourceType": "Observation", - "id": "8a95619d-84a9-41ea-aca9-ec18f5b74f73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 8.663, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4754fea8-3cde-40d2-9e69-0e960a45b01f", - "resource": { - "resourceType": "Observation", - "id": "4754fea8-3cde-40d2-9e69-0e960a45b01f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 4.1903, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbf04538-bb3f-4ad6-abd9-d71470327d11", - "resource": { - "resourceType": "Observation", - "id": "bbf04538-bb3f-4ad6-abd9-d71470327d11", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 14.815, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6ea04164-003b-4a17-9912-8a1dc3da6851", - "resource": { - "resourceType": "Observation", - "id": "6ea04164-003b-4a17-9912-8a1dc3da6851", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 49.421, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b5c8fa29-9df2-4800-b84e-bead29dc9cff", - "resource": { - "resourceType": "Observation", - "id": "b5c8fa29-9df2-4800-b84e-bead29dc9cff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 82.277, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6edb1ce-7e50-44bc-9557-9f390bc97da5", - "resource": { - "resourceType": "Observation", - "id": "c6edb1ce-7e50-44bc-9557-9f390bc97da5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 30.975, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0acf358c-881e-43d5-8e1f-8ff090e31723", - "resource": { - "resourceType": "Observation", - "id": "0acf358c-881e-43d5-8e1f-8ff090e31723", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 33.195, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12b0aabe-4522-474b-a0d2-6ec95ce0b10d", - "resource": { - "resourceType": "Observation", - "id": "12b0aabe-4522-474b-a0d2-6ec95ce0b10d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 41.104, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1425323-297b-46b9-90d6-98a36e912771", - "resource": { - "resourceType": "Observation", - "id": "d1425323-297b-46b9-90d6-98a36e912771", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 233.63, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b080b4fa-ce5c-4187-b36e-6f517dfe44e8", - "resource": { - "resourceType": "Observation", - "id": "b080b4fa-ce5c-4187-b36e-6f517dfe44e8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 167.11, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a99738d3-4ff9-4154-b055-2e8b204fb676", - "resource": { - "resourceType": "Observation", - "id": "a99738d3-4ff9-4154-b055-2e8b204fb676", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueQuantity": { - "value": 11.285, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2bfaa421-4b59-45ea-924a-e41c5268378e", - "resource": { - "resourceType": "Observation", - "id": "2bfaa421-4b59-45ea-924a-e41c5268378e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eb5f18a0-e47c-468b-ba6b-6321b8bd6b02", - "resource": { - "resourceType": "Immunization", - "id": "eb5f18a0-e47c-468b-ba6b-6321b8bd6b02", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "occurrenceDateTime": "2019-05-26T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:24291062-f162-49bc-8b23-c6483ecf3dac", - "resource": { - "resourceType": "DiagnosticReport", - "id": "24291062-f162-49bc-8b23-c6483ecf3dac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v2-0074", - "code": "LAB", - "display": "Laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - }, - "effectiveDateTime": "2019-05-26T17:32:38-07:00", - "issued": "2019-05-26T17:32:38.789-07:00", - "result": [ - { - "reference": "urn:uuid:8a95619d-84a9-41ea-aca9-ec18f5b74f73", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:4754fea8-3cde-40d2-9e69-0e960a45b01f", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:bbf04538-bb3f-4ad6-abd9-d71470327d11", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:6ea04164-003b-4a17-9912-8a1dc3da6851", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:b5c8fa29-9df2-4800-b84e-bead29dc9cff", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:c6edb1ce-7e50-44bc-9557-9f390bc97da5", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:0acf358c-881e-43d5-8e1f-8ff090e31723", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:12b0aabe-4522-474b-a0d2-6ec95ce0b10d", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:d1425323-297b-46b9-90d6-98a36e912771", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b080b4fa-ce5c-4187-b36e-6f517dfe44e8", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:a99738d3-4ff9-4154-b055-2e8b204fb676", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7cdd17aa-39d6-4423-add1-22b9bf6e0dab", - "resource": { - "resourceType": "Claim", - "id": "7cdd17aa-39d6-4423-add1-22b9bf6e0dab", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2019-05-26T17:32:38-07:00", - "end": "2019-05-26T17:47:38-07:00" - }, - "created": "2019-05-26T17:47:38-07:00", - "provider": { - "reference": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "display": "PCP83180" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:eb5f18a0-e47c-468b-ba6b-6321b8bd6b02" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5774b0f5-269e-4d39-8a5b-8aae6726f0e4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5774b0f5-269e-4d39-8a5b-8aae6726f0e4", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "performer": [ - { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7cdd17aa-39d6-4423-add1-22b9bf6e0dab" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2019-05-26T17:47:38-07:00", - "end": "2020-05-26T17:47:38-07:00" - }, - "created": "2019-05-26T17:47:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:7cdd17aa-39d6-4423-add1-22b9bf6e0dab" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - }, - "servicedPeriod": { - "start": "2019-05-26T17:32:38-07:00", - "end": "2019-05-26T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f3d3fd85-ab22-43fb-8c38-d25621657a04" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2019-05-26T17:32:38-07:00", - "end": "2019-05-26T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "19", - "display": "Off Campus-Outpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1", - "resource": { - "resourceType": "Encounter", - "id": "4969b899-5b8e-4f3b-8196-6b8a2315aeb1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", - "resource": { - "resourceType": "Condition", - "id": "cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", - "clinicalStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-clinical", - "code": "resolved" - } - ] - }, - "verificationStatus": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/condition-ver-status", - "code": "confirmed" - } - ] - }, - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - }, - "onsetDateTime": "2020-03-13T17:32:38-07:00", - "abatementDateTime": "2020-03-27T17:32:38-07:00", - "recordedDate": "2020-03-13T17:32:38-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:de0b0703-7f5a-4e1d-8ad3-eb085a00801a", - "resource": { - "resourceType": "Procedure", - "id": "de0b0703-7f5a-4e1d-8ad3-eb085a00801a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - }, - "performedPeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T17:45:38-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562", - "display": "Acute bronchitis (disorder)" - }, - { - "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc", - "display": "Acute bronchitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9", - "resource": { - "resourceType": "MedicationRequest", - "id": "c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - }, - "authoredOn": "2020-03-13T17:32:38-07:00", - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - }, - "reasonReference": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - }, - { - "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:06e84f61-f998-48c5-8aeb-dc5dca4786be", - "resource": { - "resourceType": "Claim", - "id": "06e84f61-f998-48c5-8aeb-dc5dca4786be", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "pharmacy" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "created": "2020-03-13T18:00:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "prescription": { - "reference": "urn:uuid:c4ab42de-a6e6-4d22-bd42-8cf47ccedeb9" - }, - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - } - ] - } - ], - "total": { - "value": 7.85, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:025332c4-e014-4146-bf51-47ebcdb8bfdc", - "resource": { - "resourceType": "CareTeam", - "id": "025332c4-e014-4146-bf51-47ebcdb8bfdc", - "status": "active", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - }, - "period": { - "start": "2020-03-13T17:32:38-07:00" - }, - "participant": [ - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "116153009", - "display": "Patient" - } - ], - "text": "Patient" - } - ], - "member": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "223366009", - "display": "Healthcare professional (occupation)" - } - ], - "text": "Healthcare professional (occupation)" - } - ], - "member": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - }, - { - "role": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "224891009", - "display": "Healthcare services (qualifier value)" - } - ], - "text": "Healthcare services (qualifier value)" - } - ], - "member": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - } - ], - "reasonCode": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - ], - "managingOrganization": [ - { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - ] - }, - "request": { - "method": "POST", - "url": "CareTeam" - } - }, - { - "fullUrl": "urn:uuid:042d6c6e-8e0d-46b1-ada1-c9cbe45f959f", - "resource": { - "resourceType": "CarePlan", - "id": "042d6c6e-8e0d-46b1-ada1-c9cbe45f959f", - "text": { - "status": "generated", - "div": "
    Care Plan for Respiratory therapy.
    Activities:
    • Respiratory therapy
    • Respiratory therapy

    Care plan is meant to treat Acute bronchitis (disorder).
    " - }, - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - }, - "period": { - "start": "2020-03-13T17:32:38-07:00" - }, - "careTeam": [ - { - "reference": "urn:uuid:025332c4-e014-4146-bf51-47ebcdb8bfdc" - } - ], - "addresses": [ - { - "reference": "urn:uuid:dcc3597f-478f-48f3-ac83-cc9767665562" - }, - { - "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "in-progress", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "in-progress", - "location": { - "display": "MOUNT AUBURN HOSPITAL" - } - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:25f7923e-88e7-4249-8e74-b0f07da9e00c", - "resource": { - "resourceType": "Claim", - "id": "25f7923e-88e7-4249-8e74-b0f07da9e00c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "created": "2020-03-13T18:00:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:de0b0703-7f5a-4e1d-8ad3-eb085a00801a" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "encounter": [ - { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - } - }, - { - "sequence": 3, - "procedureSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "net": { - "value": 7186.11, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7178d1d7-f7c6-458e-a7a6-212eb707fbbb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7178d1d7-f7c6-458e-a7a6-212eb707fbbb", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "Humana" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "Humana" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "25f7923e-88e7-4249-8e74-b0f07da9e00c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2020-03-13T18:00:38-07:00", - "end": "2021-03-13T18:00:38-08:00" - }, - "created": "2020-03-13T18:00:38-07:00", - "insurer": { - "display": "Humana" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:25f7923e-88e7-4249-8e74-b0f07da9e00c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:cc173d5d-7a32-44a9-b6ed-c58cf9b888bc" - }, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-diagnosistype", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "Humana" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - }, - "servicedPeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4969b899-5b8e-4f3b-8196-6b8a2315aeb1" - } - ] - }, - { - "sequence": 2, - "diagnosisSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "servicedPeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "servicedPeriod": { - "start": "2020-03-13T17:32:38-07:00", - "end": "2020-03-13T18:00:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 7186.11, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1437.222, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 5748.888, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 7186.11, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 7186.11, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 5748.888, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0", - "resource": { - "resourceType": "Encounter", - "id": "59e9b993-6536-4a29-94b4-06b0ac1c47d0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Mr. Seymour882 Shanahan202" - }, - "participant": [ - { - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/v3-ParticipationType", - "code": "PPRF", - "display": "primary performer" - } - ], - "text": "primary performer" - } - ], - "period": { - "start": "2020-03-22T17:32:38-07:00", - "end": "2020-03-22T17:47:38-07:00" - }, - "individual": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "display": "Dr. Ja391 Streich926" - } - } - ], - "period": { - "start": "2020-03-22T17:32:38-07:00", - "end": "2020-03-22T17:47:38-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2f0a548b-44dd-46d6-bd8e-c335c6f7598d", - "resource": { - "resourceType": "Observation", - "id": "2f0a548b-44dd-46d6-bd8e-c335c6f7598d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 170.3, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aa117c1a-2047-4635-b705-09096602ac2c", - "resource": { - "resourceType": "Observation", - "id": "aa117c1a-2047-4635-b705-09096602ac2c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ef26cfb-f9d7-4fb5-b827-a36c4e4cf244", - "resource": { - "resourceType": "Observation", - "id": "5ef26cfb-f9d7-4fb5-b827-a36c4e4cf244", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 82.9, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0fa9f9af-3fb7-4f3f-82b8-abe91edbc464", - "resource": { - "resourceType": "Observation", - "id": "0fa9f9af-3fb7-4f3f-82b8-abe91edbc464", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 28.59, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b89f5128-bafe-4f0e-830b-5ca840622f5b", - "resource": { - "resourceType": "Observation", - "id": "b89f5128-bafe-4f0e-830b-5ca840622f5b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:73d21f72-87f9-43ce-a273-ec51c563d673", - "resource": { - "resourceType": "Observation", - "id": "73d21f72-87f9-43ce-a273-ec51c563d673", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b696278a-7117-4c92-96db-a217891fa3bb", - "resource": { - "resourceType": "Observation", - "id": "b696278a-7117-4c92-96db-a217891fa3bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fdc15f45-8944-40c8-ac25-b68398ea4e01", - "resource": { - "resourceType": "Observation", - "id": "fdc15f45-8944-40c8-ac25-b68398ea4e01", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "effectiveDateTime": "2020-03-22T17:32:38-07:00", - "issued": "2020-03-22T17:32:38.789-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:46fab8e3-cde4-4dcf-a0e5-d814380ce826", - "resource": { - "resourceType": "Immunization", - "id": "46fab8e3-cde4-4dcf-a0e5-d814380ce826", - "status": "completed", - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "encounter": { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - }, - "occurrenceDateTime": "2020-03-22T17:32:38-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b8b07ad6-eae4-4e45-b82b-a30d4836203c", - "resource": { - "resourceType": "Claim", - "id": "b8b07ad6-eae4-4e45-b82b-a30d4836203c", - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3", - "display": "Seymour882 Shanahan202" - }, - "billablePeriod": { - "start": "2020-03-22T17:32:38-07:00", - "end": "2020-03-22T17:47:38-07:00" - }, - "created": "2020-03-22T17:47:38-07:00", - "provider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "display": "MOUNT AUBURN HOSPITAL" - }, - "priority": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/processpriority", - "code": "normal" - } - ] - }, - "supportingInfo": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:46fab8e3-cde4-4dcf-a0e5-d814380ce826" - } - } - ], - "insurance": [ - { - "sequence": 1, - "focal": true, - "coverage": { - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "encounter": [ - { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "net": { - "value": 140.52, - "currency": "USD" - } - } - ], - "total": { - "value": 129.16, - "currency": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d11b0238-af92-47c0-b1c7-a1fee525262c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d11b0238-af92-47c0-b1c7-a1fee525262c", - "contained": [ - { - "resourceType": "ServiceRequest", - "id": "referral", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "requester": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "performer": [ - { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "status": "active", - "type": { - "text": "NO_INSURANCE" - }, - "beneficiary": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "payor": [ - { - "display": "NO_INSURANCE" - } - ] - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b8b07ad6-eae4-4e45-b82b-a30d4836203c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claim-type", - "code": "institutional" - } - ] - }, - "use": "claim", - "patient": { - "reference": "urn:uuid:55a5307d-5f23-49c1-9100-7d5c513abca3" - }, - "billablePeriod": { - "start": "2020-03-22T17:47:38-07:00", - "end": "2021-03-22T17:47:38-07:00" - }, - "created": "2020-03-22T17:47:38-07:00", - "insurer": { - "display": "NO_INSURANCE" - }, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "referral": { - "reference": "#referral" - }, - "claim": { - "reference": "urn:uuid:b8b07ad6-eae4-4e45-b82b-a30d4836203c" - }, - "outcome": "complete", - "careTeam": [ - { - "sequence": 1, - "provider": { - "reference": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578" - }, - "role": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": [ - { - "focal": true, - "coverage": { - "reference": "#coverage", - "display": "NO_INSURANCE" - } - } - ], - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - }, - "servicedPeriod": { - "start": "2020-03-22T17:32:38-07:00", - "end": "2020-03-22T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:59e9b993-6536-4a29-94b4-06b0ac1c47d0" - } - ] - }, - { - "sequence": 2, - "informationSequence": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "productOrService": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "servicedPeriod": { - "start": "2020-03-22T17:32:38-07:00", - "end": "2020-03-22T17:47:38-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/ex-serviceplace", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "currency": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "currency": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "total": [ - { - "category": { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/adjudication", - "code": "submitted", - "display": "Submitted Amount" - } - ], - "text": "Submitted Amount" - }, - "amount": { - "value": 129.16, - "currency": "USD" - } - } - ], - "payment": { - "amount": { - "value": 112.41600000000001, - "currency": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json deleted file mode 100644 index 0584f07f2676..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/hospitalInformation1586368892823.json +++ /dev/null @@ -1,1182 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 20 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "resource": { - "resourceType": "Organization", - "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 4 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAMBRIDGE HEALTH ALLIANCE", - "telecom": [ - { - "system": "phone", - "value": "6176652300" - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:ef6ab57c-ed94-3dbe-9861-812d515918b3", - "resource": { - "resourceType": "Organization", - "id": "ef6ab57c-ed94-3dbe-9861-812d515918b3", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 199 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ef6ab57c-ed94-3dbe-9861-812d515918b3" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAPE COD HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5087711800" - } - ], - "address": [ - { - "line": [ - "88 LEWIS BAY ROAD" - ], - "city": "HYANNIS", - "state": "MA", - "postalCode": "02601", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:b0e04623-b02c-3f8b-92ea-943fc4db60da", - "resource": { - "resourceType": "Organization", - "id": "b0e04623-b02c-3f8b-92ea-943fc4db60da", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 38 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "b0e04623-b02c-3f8b-92ea-943fc4db60da" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "LOWELL GENERAL HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "9789376000" - } - ], - "address": [ - { - "line": [ - "295 VARNUM AVENUE" - ], - "city": "LOWELL", - "state": "MA", - "postalCode": "01854", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d78e84ec-30aa-3bba-a33a-f29a3a454662", - "resource": { - "resourceType": "Organization", - "id": "d78e84ec-30aa-3bba-a33a-f29a3a454662", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 10 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d78e84ec-30aa-3bba-a33a-f29a3a454662" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MASSACHUSETTS GENERAL HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6177262000" - } - ], - "address": [ - { - "line": [ - "55 FRUIT STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:23834663-ed53-3da9-b330-d6e1ecb8428e", - "resource": { - "resourceType": "Organization", - "id": "23834663-ed53-3da9-b330-d6e1ecb8428e", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 12 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "23834663-ed53-3da9-b330-d6e1ecb8428e" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SOUTHCOAST HOSPITAL GROUP, INC", - "telecom": [ - { - "system": "phone", - "value": "5086793131" - } - ], - "address": [ - { - "line": [ - "363 HIGHLAND AVENUE" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02720", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:44bef9d3-91c2-3005-93e0-ccf436348ff0", - "resource": { - "resourceType": "Organization", - "id": "44bef9d3-91c2-3005-93e0-ccf436348ff0", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 29 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "44bef9d3-91c2-3005-93e0-ccf436348ff0" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MASSACHUSETTS EYE AND EAR INFIRMARY -", - "telecom": [ - { - "system": "phone", - "value": "6175237900" - } - ], - "address": [ - { - "line": [ - "243 CHARLES STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d733d4a9-080d-3593-b910-2366e652b7ea", - "resource": { - "resourceType": "Organization", - "id": "d733d4a9-080d-3593-b910-2366e652b7ea", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 79 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d733d4a9-080d-3593-b910-2366e652b7ea" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "BRIGHAM AND WOMEN'S FAULKNER HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6179837000" - } - ], - "address": [ - { - "line": [ - "1153 CENTRE STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02130", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:ff9863d3-3fa3-3861-900e-f00148f5d9c2", - "resource": { - "resourceType": "Organization", - "id": "ff9863d3-3fa3-3861-900e-f00148f5d9c2", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "ff9863d3-3fa3-3861-900e-f00148f5d9c2" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SHRINERS' HOSPITAL FOR CHILDREN - BOSTON, THE", - "telecom": [ - { - "system": "phone", - "value": "6177223000" - } - ], - "address": [ - { - "line": [ - "51 BLOSSOM STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "resource": { - "resourceType": "Organization", - "id": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 42 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 38 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 30 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 15 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "35a49dfd-7ce3-3ef2-bbb3-eab67a5f26c3" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "NEW ENGLAND ORAL SURGERY ASSOCIATES LLC", - "telecom": [ - { - "system": "phone", - "value": "978-667-8600" - } - ], - "address": [ - { - "line": [ - "2 ANDOVER RD" - ], - "city": "BILLERICA", - "state": "MA", - "postalCode": "01821-1916", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "resource": { - "resourceType": "Organization", - "id": "7f7c08c8-1c39-3b26-bd34-5122a003dd91", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 42 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 29 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 16 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 18 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "7f7c08c8-1c39-3b26-bd34-5122a003dd91" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP83180", - "telecom": [ - { - "system": "phone", - "value": "617-389-6951" - } - ], - "address": [ - { - "line": [ - "107 FERRY ST" - ], - "city": "EVERETT", - "state": "MA", - "postalCode": "02149-4940", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "resource": { - "resourceType": "Organization", - "id": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 81 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 515 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 85 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 104 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e9e2998f-97d3-3f88-8ee8-82660c5ddfce" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP128586", - "address": [ - { - "line": [ - "2360 CRANBERRY HWY" - ], - "city": "W WAREHAM", - "state": "MA", - "postalCode": "02576-1208", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "resource": { - "resourceType": "Organization", - "id": "75bdb2ec-cbd0-35af-b0fe-18b111890b67", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 25 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 15 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 4 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 8 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "75bdb2ec-cbd0-35af-b0fe-18b111890b67" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP235727", - "address": [ - { - "line": [ - "67 SLADES FERRY BLVD" - ], - "city": "SOMERSET", - "state": "MA", - "postalCode": "02726", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "resource": { - "resourceType": "Organization", - "id": "76a80350-916e-3fc9-8fed-2c8c4b36c42f", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 54 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 321 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 48 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 55 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "76a80350-916e-3fc9-8fed-2c8c4b36c42f" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "NORTH SUFFOLK MENTAL HEALTH ASSOCIATION, INC.", - "telecom": [ - { - "system": "phone", - "value": "617-912-7900" - } - ], - "address": [ - { - "line": [ - "301 BROADWAY" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "02150-2807", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "resource": { - "resourceType": "Organization", - "id": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "4feb5860-8c43-34ea-b2c8-84ff7ca8740b" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "TRU MEDICAL WALK IN CENTER", - "telecom": [ - { - "system": "phone", - "value": "508-675-1522" - } - ], - "address": [ - { - "line": [ - "528 NEWTON STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "2721", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:7116c200-6663-3503-8314-1ee16845e5d3", - "resource": { - "resourceType": "Organization", - "id": "7116c200-6663-3503-8314-1ee16845e5d3", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "7116c200-6663-3503-8314-1ee16845e5d3" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SAINTS WALK-IN MEDICAL CENTER - URGENT CARE AND OCCUPATIONAL HEALTH", - "telecom": [ - { - "system": "phone", - "value": "978-458-6868" - } - ], - "address": [ - { - "line": [ - "85 PARKHURST ROAD" - ], - "city": "CHELMSFORD", - "state": "MA", - "postalCode": "1824", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "resource": { - "resourceType": "Organization", - "id": "3265f387-6c51-32ee-8f6d-b2a89caa34d5", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "3265f387-6c51-32ee-8f6d-b2a89caa34d5" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CHELSEA MGH HEALTH CENTER MEDICAL WALK IN", - "telecom": [ - { - "system": "phone", - "value": "617-884-8300" - } - ], - "address": [ - { - "line": [ - "151 EVERETT AVENUE" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "2150", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:0f7f97f6-693c-3de9-8190-85feaf4632e2", - "resource": { - "resourceType": "Organization", - "id": "0f7f97f6-693c-3de9-8190-85feaf4632e2", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "0f7f97f6-693c-3de9-8190-85feaf4632e2" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CONCENTRA URGENT CARE - EAST BOSTON", - "telecom": [ - { - "system": "phone", - "value": "617-568-6500" - } - ], - "address": [ - { - "line": [ - "1 HARBORSIDE DRIVE" - ], - "city": "EAST BOSTON", - "state": "MA", - "postalCode": "2128", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json b/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json deleted file mode 100644 index c2d6b7250129..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/R4/practitionerInformation1586368892823.json +++ /dev/null @@ -1,978 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:eabb178d-5fde-3f2a-b2b0-84b601205578", - "resource": { - "resourceType": "Practitioner", - "id": "eabb178d-5fde-3f2a-b2b0-84b601205578", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 20 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999989" - } - ], - "active": true, - "name": [ - { - "family": "Streich926", - "given": [ - "Ja391" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Ja391.Streich926@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "resource": { - "resourceType": "Practitioner", - "id": "e4e8b8e3-604e-3bca-978b-afcfa63491d0", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 4 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999959" - } - ], - "active": true, - "name": [ - { - "family": "Harvey63", - "given": [ - "Alla648" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Alla648.Harvey63@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:fb3b949e-e3cb-3069-b280-521ac0512d2e", - "resource": { - "resourceType": "Practitioner", - "id": "fb3b949e-e3cb-3069-b280-521ac0512d2e", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 199 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999949" - } - ], - "active": true, - "name": [ - { - "family": "Kautzer186", - "given": [ - "Heath320" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Heath320.Kautzer186@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "88 LEWIS BAY ROAD" - ], - "city": "HYANNIS", - "state": "MA", - "postalCode": "02601", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "resource": { - "resourceType": "Practitioner", - "id": "57adca03-a2e1-3753-bcc5-e6ef86bbb821", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 38 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999759" - } - ], - "active": true, - "name": [ - { - "family": "Tillman293", - "given": [ - "Franklin857" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Franklin857.Tillman293@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "295 VARNUM AVENUE" - ], - "city": "LOWELL", - "state": "MA", - "postalCode": "01854", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:7b0a96ca-9d87-3146-96e7-ef4e67128176", - "resource": { - "resourceType": "Practitioner", - "id": "7b0a96ca-9d87-3146-96e7-ef4e67128176", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 10 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999719" - } - ], - "active": true, - "name": [ - { - "family": "Oberbrunner298", - "given": [ - "Herlinda751" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Herlinda751.Oberbrunner298@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "55 FRUIT STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:f411d1de-1d0c-365f-8a71-8d225cb962af", - "resource": { - "resourceType": "Practitioner", - "id": "f411d1de-1d0c-365f-8a71-8d225cb962af", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 12 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999699" - } - ], - "active": true, - "name": [ - { - "family": "Brakus656", - "given": [ - "Gillian484" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Gillian484.Brakus656@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "363 HIGHLAND AVENUE" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "02720", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "resource": { - "resourceType": "Practitioner", - "id": "fbdf1eef-68c6-378d-90ac-b746ba9bd7ad", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 29 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999689" - } - ], - "active": true, - "name": [ - { - "family": "Mayert710", - "given": [ - "Dovie983" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Dovie983.Mayert710@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "243 CHARLES STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:3a576225-3a87-3704-a672-e1c8bc03389f", - "resource": { - "resourceType": "Practitioner", - "id": "3a576225-3a87-3704-a672-e1c8bc03389f", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 79 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999519" - } - ], - "active": true, - "name": [ - { - "family": "Howell947", - "given": [ - "Max124" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Max124.Howell947@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "1153 CENTRE STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02130", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:4c2aa782-3c72-3d1c-be80-70a3bae80c38", - "resource": { - "resourceType": "Practitioner", - "id": "4c2aa782-3c72-3d1c-be80-70a3bae80c38", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999999379" - } - ], - "active": true, - "name": [ - { - "family": "Graham902", - "given": [ - "Spencer878" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Spencer878.Graham902@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "51 BLOSSOM STREET" - ], - "city": "BOSTON", - "state": "MA", - "postalCode": "02114", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:0d5ab589-1d26-346a-b8db-e425d2e139e7", - "resource": { - "resourceType": "Practitioner", - "id": "0d5ab589-1d26-346a-b8db-e425d2e139e7", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 42 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999969899" - } - ], - "active": true, - "name": [ - { - "family": "Gerlach374", - "given": [ - "Lillia547" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Lillia547.Gerlach374@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "2 ANDOVER RD" - ], - "city": "BILLERICA", - "state": "MA", - "postalCode": "01821-1916", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:12810b32-78af-32f0-b85c-8177e3f2e2e2", - "resource": { - "resourceType": "Practitioner", - "id": "12810b32-78af-32f0-b85c-8177e3f2e2e2", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 42 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999959759" - } - ], - "active": true, - "name": [ - { - "family": "Goldner995", - "given": [ - "Doretta917" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Doretta917.Goldner995@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "107 FERRY ST" - ], - "city": "EVERETT", - "state": "MA", - "postalCode": "02149-4940", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:cf99bf43-51e5-3bab-8623-211cd48f66dc", - "resource": { - "resourceType": "Practitioner", - "id": "cf99bf43-51e5-3bab-8623-211cd48f66dc", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 81 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999951739" - } - ], - "active": true, - "name": [ - { - "family": "DuBuque211", - "given": [ - "Carlyn477" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Carlyn477.DuBuque211@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "2360 CRANBERRY HWY" - ], - "city": "W WAREHAM", - "state": "MA", - "postalCode": "02576-1208", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "resource": { - "resourceType": "Practitioner", - "id": "b5c349b9-6f37-31da-a6a0-4c9e00e11ad8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 25 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999931859" - } - ], - "active": true, - "name": [ - { - "family": "Mante251", - "given": [ - "Bertie593" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Bertie593.Mante251@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "67 SLADES FERRY BLVD" - ], - "city": "SOMERSET", - "state": "MA", - "postalCode": "02726", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:0390078a-1894-335b-9136-5f518ee1dbb6", - "resource": { - "resourceType": "Practitioner", - "id": "0390078a-1894-335b-9136-5f518ee1dbb6", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 54 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999931799" - } - ], - "active": true, - "name": [ - { - "family": "Koelpin146", - "given": [ - "Cheryle584" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Cheryle584.Koelpin146@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "301 BROADWAY" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "02150-2807", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:21a54030-cce5-3104-a773-9eb9bd60830b", - "resource": { - "resourceType": "Practitioner", - "id": "21a54030-cce5-3104-a773-9eb9bd60830b", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906799" - } - ], - "active": true, - "name": [ - { - "family": "Eichmann909", - "given": [ - "Wanetta360" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Wanetta360.Eichmann909@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "528 NEWTON STREET" - ], - "city": "FALL RIVER", - "state": "MA", - "postalCode": "2721", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:389e216f-1825-347e-b23b-630d2f37b195", - "resource": { - "resourceType": "Practitioner", - "id": "389e216f-1825-347e-b23b-630d2f37b195", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906719" - } - ], - "active": true, - "name": [ - { - "family": "Watsica258", - "given": [ - "Gidget756" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Gidget756.Watsica258@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "85 PARKHURST ROAD" - ], - "city": "CHELMSFORD", - "state": "MA", - "postalCode": "1824", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:fe82a096-86b9-344b-b405-bd24b6ad4715", - "resource": { - "resourceType": "Practitioner", - "id": "fe82a096-86b9-344b-b405-bd24b6ad4715", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906709" - } - ], - "active": true, - "name": [ - { - "family": "Hoppe518", - "given": [ - "Devin82" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Devin82.Hoppe518@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "151 EVERETT AVENUE" - ], - "city": "CHELSEA", - "state": "MA", - "postalCode": "2150", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:1296d3c9-02e0-34a2-86e1-0e185a8f92d6", - "resource": { - "resourceType": "Practitioner", - "id": "1296d3c9-02e0-34a2-86e1-0e185a8f92d6", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "9999906459" - } - ], - "active": true, - "name": [ - { - "family": "Ankunding277", - "given": [ - "Elton404" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "system": "email", - "value": "Elton404.Ankunding277@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "1 HARBORSIDE DRIVE" - ], - "city": "EAST BOSTON", - "state": "MA", - "postalCode": "2128", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json deleted file mode 100644 index 986c874a86e7..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Alexander630_Romaguera67_02a63c07-9fcc-42ba-aec0-9d5399ac4796.json +++ /dev/null @@ -1,16003 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796", - "resource": { - "resourceType": "Patient", - "id": "02a63c07-9fcc-42ba-aec0-9d5399ac4796", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 4798728652809701405 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2054-5", - "display": "Black or African American" - } - }, - { - "url": "text", - "valueString": "Black or African American" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Mina319 Langworth352" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "M" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Winchester", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.0805149386188746 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 0.9194850613811254 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-16-7591" - } - ], - "name": [ - { - "use": "official", - "family": "Romaguera67", - "given": [ - "Alexander630" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-213-6346", - "use": "home" - } - ], - "gender": "male", - "birthDate": "2018-07-02", - "deceasedDateTime": "2020-04-13T06:13:10-07:00", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.35034978194894 - }, - { - "url": "longitude", - "valueDecimal": -71.11270110210596 - } - ] - } - ], - "line": [ - "639 Rogahn Grove" - ], - "city": "Cambridge", - "state": "Massachusetts", - "postalCode": "02138", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5", - "resource": { - "resourceType": "Organization", - "id": "aced4ce4-957a-3d86-b20a-302d12f949a5", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "LONGFELLOW PRIMARY CARE, PC", - "telecom": [ - { - "system": "phone", - "value": "617-492-4545" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.376043 - }, - { - "url": "longitude", - "valueDecimal": -71.11868 - } - ] - } - ], - "line": [ - "625 MOUNT AUBURN ST" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138-4518", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886", - "resource": { - "resourceType": "Practitioner", - "id": "29e6e246-9052-309c-a47f-c2e5bc60c886", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "24070" - } - ], - "active": true, - "name": [ - { - "family": "Waelchi213", - "given": [ - "Cristopher265" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "625 MOUNT AUBURN ST" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138-4518", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492", - "resource": { - "resourceType": "Encounter", - "id": "7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fd976131-d3fc-459a-a942-2cbde9b4ded2", - "resource": { - "resourceType": "Immunization", - "id": "fd976131-d3fc-459a-a942-2cbde9b4ded2", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492" - }, - "date": "2018-07-02T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ac6048bf-c392-489e-a72c-fb2b13c1c181", - "resource": { - "resourceType": "Claim", - "id": "ac6048bf-c392-489e-a72c-fb2b13c1c181", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:fd976131-d3fc-459a-a942-2cbde9b4ded2" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:7ad16f28-8d6f-4b2d-a0f2-34ada4ef4492" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:750f8437-b065-4c2b-bb4b-05b8b1a90ed8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "750f8437-b065-4c2b-bb4b-05b8b1a90ed8", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ac6048bf-c392-489e-a72c-fb2b13c1c181" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-07-02T06:28:10-07:00", - "end": "2019-07-02T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", - "resource": { - "resourceType": "Practitioner", - "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10" - } - ], - "active": true, - "name": [ - { - "family": "Jenkins714", - "given": [ - "Lara964" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47", - "resource": { - "resourceType": "Encounter", - "id": "b0454769-6ade-48ac-90f1-2ab20e998a47", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency Room Admission" - } - ], - "text": "Emergency Room Admission" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "128613002", - "display": "Seizure disorder" - } - ] - } - ], - "hospitalization": { - "dischargeDisposition": { - "coding": [ - { - "system": "http://www.nubc.org/patient-discharge", - "code": "01", - "display": "Discharged to home care or self care (routine discharge)" - } - ], - "text": "Discharged to home care or self care (routine discharge)" - } - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92", - "resource": { - "resourceType": "Condition", - "id": "317bc0bf-3dc2-4233-8b6f-3540d1bc9e92", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "128613002", - "display": "Seizure disorder" - } - ], - "text": "Seizure disorder" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "onsetDateTime": "2018-07-02T06:13:10-07:00", - "assertedDate": "2018-07-02T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113", - "resource": { - "resourceType": "Condition", - "id": "8c6f2229-a6ab-4397-ace4-b4ecaeb78113", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "703151001", - "display": "History of single seizure (situation)" - } - ], - "text": "History of single seizure (situation)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "onsetDateTime": "2018-07-02T06:13:10-07:00", - "assertedDate": "2018-07-02T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:449ade9a-34c5-4e70-a39b-df8ad1f3f87c", - "resource": { - "resourceType": "Observation", - "id": "449ade9a-34c5-4e70-a39b-df8ad1f3f87c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 52.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84d0c337-8875-4c8c-adff-06590ae484d8", - "resource": { - "resourceType": "Observation", - "id": "84d0c337-8875-4c8c-adff-06590ae484d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e1cdff5b-fd65-45b9-9371-64fd2eba01ed", - "resource": { - "resourceType": "Observation", - "id": "e1cdff5b-fd65-45b9-9371-64fd2eba01ed", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 3.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49baefdd-a2f6-4af6-b819-6a9ecbceb309", - "resource": { - "resourceType": "Observation", - "id": "49baefdd-a2f6-4af6-b819-6a9ecbceb309", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.11714, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:800e4f80-0a10-4b4c-84a4-4287fec2b32a", - "resource": { - "resourceType": "Observation", - "id": "800e4f80-0a10-4b4c-84a4-4287fec2b32a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 35.720, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:acd19516-ac3e-48f8-92dd-3eddfdce847e", - "resource": { - "resourceType": "Observation", - "id": "acd19516-ac3e-48f8-92dd-3eddfdce847e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:187c0917-46dd-4849-b48f-b7736c147be4", - "resource": { - "resourceType": "Observation", - "id": "187c0917-46dd-4849-b48f-b7736c147be4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 80, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd47a3de-5003-4dda-bf22-d436d8bb34de", - "resource": { - "resourceType": "Observation", - "id": "dd47a3de-5003-4dda-bf22-d436d8bb34de", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bc594c64-c12c-4901-b534-efd4cc16812a", - "resource": { - "resourceType": "Observation", - "id": "bc594c64-c12c-4901-b534-efd4cc16812a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 8.1663, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:507829cc-ba3d-481b-8739-481dbb4971da", - "resource": { - "resourceType": "Observation", - "id": "507829cc-ba3d-481b-8739-481dbb4971da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 5.2914, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7b8b65d-2008-4e69-b4ad-761096d58385", - "resource": { - "resourceType": "Observation", - "id": "b7b8b65d-2008-4e69-b4ad-761096d58385", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 15.174, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a92e98ac-976f-4f7c-af03-ef6684e9f7e9", - "resource": { - "resourceType": "Observation", - "id": "a92e98ac-976f-4f7c-af03-ef6684e9f7e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 38.942, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", - "resource": { - "resourceType": "Observation", - "id": "ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 83.049, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", - "resource": { - "resourceType": "Observation", - "id": "b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 28.367, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f0944211-bbcd-46b9-9349-0a3e754b1bc7", - "resource": { - "resourceType": "Observation", - "id": "f0944211-bbcd-46b9-9349-0a3e754b1bc7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 35.103, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:74983c05-e0b0-4b59-9a17-735d84341aa2", - "resource": { - "resourceType": "Observation", - "id": "74983c05-e0b0-4b59-9a17-735d84341aa2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 43.812, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cdfef0be-8691-4652-aaa9-0cbeee650aa9", - "resource": { - "resourceType": "Observation", - "id": "cdfef0be-8691-4652-aaa9-0cbeee650aa9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 243.61, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7910aa61-9b39-4967-8a11-c70d76cbc1c1", - "resource": { - "resourceType": "Observation", - "id": "7910aa61-9b39-4967-8a11-c70d76cbc1c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 153.31, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4a4355e0-07b2-4193-960b-3d26f391d60d", - "resource": { - "resourceType": "Observation", - "id": "4a4355e0-07b2-4193-960b-3d26f391d60d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 9.8262, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e377311f-78bd-44ef-8963-8c3c8209e629", - "resource": { - "resourceType": "Observation", - "id": "e377311f-78bd-44ef-8963-8c3c8209e629", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddff5e15-83b7-484a-91fa-be6e84f0df22", - "resource": { - "resourceType": "Procedure", - "id": "ddff5e15-83b7-484a-91fa-be6e84f0df22", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "54550000", - "display": "Seizure Count Cerebral Cortex Electroencephalogram (EEG)" - } - ], - "text": "Seizure Count Cerebral Cortex Electroencephalogram (EEG)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "performedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T08:50:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:82160634-a224-42b2-b4e4-77cf9129d853", - "resource": { - "resourceType": "MedicationRequest", - "id": "82160634-a224-42b2-b4e4-77cf9129d853", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308971", - "display": "Carbamazepine[Tegretol]" - } - ], - "text": "Carbamazepine[Tegretol]" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "authoredOn": "2018-07-02T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:c7498b56-a644-480d-88dc-7d190761879b", - "resource": { - "resourceType": "Claim", - "id": "c7498b56-a644-480d-88dc-7d190761879b", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:82160634-a224-42b2-b4e4-77cf9129d853" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - } - ] - } - ], - "total": { - "value": 325.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d05cec1c-89e2-456f-9839-f39af8419f28", - "resource": { - "resourceType": "DiagnosticReport", - "id": "d05cec1c-89e2-456f-9839-f39af8419f28", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "result": [ - { - "reference": "urn:uuid:bc594c64-c12c-4901-b534-efd4cc16812a", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:507829cc-ba3d-481b-8739-481dbb4971da", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b7b8b65d-2008-4e69-b4ad-761096d58385", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:a92e98ac-976f-4f7c-af03-ef6684e9f7e9", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:ff5f1e9a-1f2a-4ec5-8d5c-12406742c829", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:b03e8247-7274-42bc-aaaf-b9c5fb6f1fc5", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:f0944211-bbcd-46b9-9349-0a3e754b1bc7", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:74983c05-e0b0-4b59-9a17-735d84341aa2", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:cdfef0be-8691-4652-aaa9-0cbeee650aa9", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:7910aa61-9b39-4967-8a11-c70d76cbc1c1", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:4a4355e0-07b2-4193-960b-3d26f391d60d", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f051abc6-d9d8-40bc-97be-e534b8b475c2", - "resource": { - "resourceType": "Claim", - "id": "f051abc6-d9d8-40bc-97be-e534b8b475c2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92" - } - }, - { - "sequence": 2, - "diagnosisReference": { - "reference": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ddff5e15-83b7-484a-91fa-be6e84f0df22" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "diagnosisLinkId": [ - 2 - ] - }, - { - "sequence": 4, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "54550000" - } - ] - }, - "net": { - "value": 12986.76, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ea48dada-6e46-4b3b-9ce9-bcd08b2e47f5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ea48dada-6e46-4b3b-9ce9-bcd08b2e47f5", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f051abc6-d9d8-40bc-97be-e534b8b475c2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-07-02T09:50:10-07:00", - "end": "2019-07-02T09:50:10-07:00" - }, - "created": "2018-07-02T09:50:10-07:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:f051abc6-d9d8-40bc-97be-e534b8b475c2" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:317bc0bf-3dc2-4233-8b6f-3540d1bc9e92" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 2, - "diagnosisReference": { - "reference": "urn:uuid:8c6f2229-a6ab-4397-ace4-b4ecaeb78113" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b0454769-6ade-48ac-90f1-2ab20e998a47" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - }, - { - "sequence": 3, - "diagnosisLinkId": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "54550000" - } - ] - }, - "servicedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T09:50:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "net": { - "value": 12986.76, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2597.3520000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 10389.408000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 12986.76, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 12986.76, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 10389.408000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12", - "resource": { - "resourceType": "Encounter", - "id": "4488ce28-6761-4741-9c26-3d8d8648af12", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2018-08-06T06:13:10-07:00", - "end": "2018-08-06T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bafc638d-077c-49e8-b900-859bdf3d6fc7", - "resource": { - "resourceType": "Observation", - "id": "bafc638d-077c-49e8-b900-859bdf3d6fc7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 56.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fec7ca8f-cbfa-4d06-bea6-093a161de5c7", - "resource": { - "resourceType": "Observation", - "id": "fec7ca8f-cbfa-4d06-bea6-093a161de5c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3cf3e132-dcfa-481e-9857-603b6ab58483", - "resource": { - "resourceType": "Observation", - "id": "3cf3e132-dcfa-481e-9857-603b6ab58483", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 4, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:971bc36f-971a-463c-bbb4-30baec5c6b13", - "resource": { - "resourceType": "Observation", - "id": "971bc36f-971a-463c-bbb4-30baec5c6b13", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.018783, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70a25cb8-f762-4b97-aaf7-04e5cdf7fe78", - "resource": { - "resourceType": "Observation", - "id": "70a25cb8-f762-4b97-aaf7-04e5cdf7fe78", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 39.130, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5ad6b974-9707-4de9-8103-41faf1e22cf0", - "resource": { - "resourceType": "Observation", - "id": "5ad6b974-9707-4de9-8103-41faf1e22cf0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 72, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e3e25ae-e1a6-43dd-875c-bb7176bb250b", - "resource": { - "resourceType": "Observation", - "id": "0e3e25ae-e1a6-43dd-875c-bb7176bb250b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 77, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6e4cc34-6827-48e2-8dbd-70679f8a0a1f", - "resource": { - "resourceType": "Observation", - "id": "e6e4cc34-6827-48e2-8dbd-70679f8a0a1f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c430effa-3a2c-497c-ad11-0f9ad5d4fb19", - "resource": { - "resourceType": "Observation", - "id": "c430effa-3a2c-497c-ad11-0f9ad5d4fb19", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:07b0a7c4-a136-4b04-b462-f62807d223c8", - "resource": { - "resourceType": "Procedure", - "id": "07b0a7c4-a136-4b04-b462-f62807d223c8", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "performedPeriod": { - "start": "2018-08-06T06:13:10-07:00", - "end": "2018-08-06T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ab69207f-d574-4100-84f0-174f94becc16", - "resource": { - "resourceType": "Immunization", - "id": "ab69207f-d574-4100-84f0-174f94becc16", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - }, - "date": "2018-08-06T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1b9403c4-b51b-4650-861d-948b9710a45a", - "resource": { - "resourceType": "Claim", - "id": "1b9403c4-b51b-4650-861d-948b9710a45a", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-08-06T06:13:10-07:00", - "end": "2018-08-06T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ab69207f-d574-4100-84f0-174f94becc16" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:07b0a7c4-a136-4b04-b462-f62807d223c8" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4488ce28-6761-4741-9c26-3d8d8648af12" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 541.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6f1c7897-af99-495d-b271-8f796fff068b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6f1c7897-af99-495d-b271-8f796fff068b", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1b9403c4-b51b-4650-861d-948b9710a45a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-08-06T06:43:10-07:00", - "end": "2019-08-06T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 108.24400000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 432.97600000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 541.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 541.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 545.392, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b", - "resource": { - "resourceType": "Encounter", - "id": "49fa8b6c-d685-441d-b10d-02ec4c92bf2b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2018-10-08T06:13:10-07:00", - "end": "2018-10-08T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5e95ade0-7a20-483b-9774-788107e3005b", - "resource": { - "resourceType": "Observation", - "id": "5e95ade0-7a20-483b-9774-788107e3005b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 62, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f26c36eb-a7aa-47a0-92e3-687f204e3b4b", - "resource": { - "resourceType": "Observation", - "id": "f26c36eb-a7aa-47a0-92e3-687f204e3b4b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:734525b0-1e22-4f51-89aa-13ca80261d11", - "resource": { - "resourceType": "Observation", - "id": "734525b0-1e22-4f51-89aa-13ca80261d11", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 5.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b4a017b1-4e7f-4901-9c13-2f56e14c64dc", - "resource": { - "resourceType": "Observation", - "id": "b4a017b1-4e7f-4901-9c13-2f56e14c64dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.42442, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:74c76fe8-f12b-4c0b-9667-427af9bf647a", - "resource": { - "resourceType": "Observation", - "id": "74c76fe8-f12b-4c0b-9667-427af9bf647a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 41.690, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d729006-4be6-4bd5-a279-7966b37d1ba5", - "resource": { - "resourceType": "Observation", - "id": "7d729006-4be6-4bd5-a279-7966b37d1ba5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d2d0467-cede-4ace-af92-861e79ff6f48", - "resource": { - "resourceType": "Observation", - "id": "2d2d0467-cede-4ace-af92-861e79ff6f48", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 67, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7ccf38c6-c989-4cea-9ea0-92715ed4860c", - "resource": { - "resourceType": "Observation", - "id": "7ccf38c6-c989-4cea-9ea0-92715ed4860c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b648e60e-bf23-46b4-b9c9-3a7a393e80ac", - "resource": { - "resourceType": "Observation", - "id": "b648e60e-bf23-46b4-b9c9-3a7a393e80ac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e04b895-97d8-4d94-a3af-3e55ad057f4f", - "resource": { - "resourceType": "Immunization", - "id": "4e04b895-97d8-4d94-a3af-3e55ad057f4f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc", - "resource": { - "resourceType": "Immunization", - "id": "fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:eabfbeb5-e820-480a-80a7-535f53e0c3b5", - "resource": { - "resourceType": "Immunization", - "id": "eabfbeb5-e820-480a-80a7-535f53e0c3b5", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a9682836-9b4b-4d94-b5e0-433fc52b6eab", - "resource": { - "resourceType": "Immunization", - "id": "a9682836-9b4b-4d94-b5e0-433fc52b6eab", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8a624fd6-4c8f-446a-a021-c0f8fee0d15b", - "resource": { - "resourceType": "Immunization", - "id": "8a624fd6-4c8f-446a-a021-c0f8fee0d15b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0f99487b-0475-43ef-9869-1fe7591c1d55", - "resource": { - "resourceType": "Claim", - "id": "0f99487b-0475-43ef-9869-1fe7591c1d55", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-10-08T06:13:10-07:00", - "end": "2018-10-08T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4e04b895-97d8-4d94-a3af-3e55ad057f4f" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:fea3fde5-fe9a-4793-8b63-0c7cb2ea21cc" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:eabfbeb5-e820-480a-80a7-535f53e0c3b5" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a9682836-9b4b-4d94-b5e0-433fc52b6eab" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8a624fd6-4c8f-446a-a021-c0f8fee0d15b" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:49fa8b6c-d685-441d-b10d-02ec4c92bf2b" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d4960c1d-5052-4012-85dd-9aa154b3238d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d4960c1d-5052-4012-85dd-9aa154b3238d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0f99487b-0475-43ef-9869-1fe7591c1d55" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-10-08T06:28:10-07:00", - "end": "2019-10-08T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d", - "resource": { - "resourceType": "Encounter", - "id": "58c3e07d-b23e-4d7c-a85e-9662620a062d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:43:10-08:00" - }, - "hospitalization": { - "dischargeDisposition": { - "coding": [ - { - "system": "http://www.nubc.org/patient-discharge", - "code": "01", - "display": "Discharged to home care or self care (routine discharge)" - } - ], - "text": "Discharged to home care or self care (routine discharge)" - } - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:442de3a6-9b3e-47aa-a449-228f2b259d22", - "resource": { - "resourceType": "Observation", - "id": "442de3a6-9b3e-47aa-a449-228f2b259d22", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 66, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d811f25-d09a-491e-9671-3677b1439524", - "resource": { - "resourceType": "Observation", - "id": "7d811f25-d09a-491e-9671-3677b1439524", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c546661-cfcb-4e91-bb0f-0b4ec8124acf", - "resource": { - "resourceType": "Observation", - "id": "7c546661-cfcb-4e91-bb0f-0b4ec8124acf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 6.6000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee5b8473-cb89-4e0e-abe5-94bef351621b", - "resource": { - "resourceType": "Observation", - "id": "ee5b8473-cb89-4e0e-abe5-94bef351621b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 3.2575, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b124abeb-b1e6-410c-ae5b-e07ada3dc650", - "resource": { - "resourceType": "Observation", - "id": "b124abeb-b1e6-410c-ae5b-e07ada3dc650", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 43.340, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:21f96e6d-9db4-4bb5-b5ca-6c70aa155358", - "resource": { - "resourceType": "Observation", - "id": "21f96e6d-9db4-4bb5-b5ca-6c70aa155358", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2e36292e-9eb2-42ab-a81d-a3c737e16fab", - "resource": { - "resourceType": "Observation", - "id": "2e36292e-9eb2-42ab-a81d-a3c737e16fab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4fbdd32a-79ab-48f0-9ac2-f728539af7bc", - "resource": { - "resourceType": "Observation", - "id": "4fbdd32a-79ab-48f0-9ac2-f728539af7bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6dde0d43-ca5b-4a01-b528-ee24df4ddf53", - "resource": { - "resourceType": "Observation", - "id": "6dde0d43-ca5b-4a01-b528-ee24df4ddf53", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecde24ef-7159-4ad5-9bdf-df56d857f7cb", - "resource": { - "resourceType": "Procedure", - "id": "ecde24ef-7159-4ad5-9bdf-df56d857f7cb", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "performedPeriod": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:28:10-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e645deb5-749a-471e-8964-0d5a3f340a20", - "resource": { - "resourceType": "MedicationRequest", - "id": "e645deb5-749a-471e-8964-0d5a3f340a20", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308971", - "display": "Carbamazepine[Tegretol]" - } - ], - "text": "Carbamazepine[Tegretol]" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "authoredOn": "2018-12-10T05:13:10-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - }, - "onBehalfOf": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:cd306f6b-9691-4503-afe1-90dc0fee38b2", - "resource": { - "resourceType": "Claim", - "id": "cd306f6b-9691-4503-afe1-90dc0fee38b2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:43:10-08:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "prescription": { - "reference": "urn:uuid:e645deb5-749a-471e-8964-0d5a3f340a20" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - } - ] - } - ], - "total": { - "value": 109.78, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:935d5e93-e972-4d27-8ab7-38afe2a51f91", - "resource": { - "resourceType": "Immunization", - "id": "935d5e93-e972-4d27-8ab7-38afe2a51f91", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:35f80ad5-66d2-49d0-a5fe-68653071b0de", - "resource": { - "resourceType": "Immunization", - "id": "35f80ad5-66d2-49d0-a5fe-68653071b0de", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4d5ae690-6eaf-41c2-bee1-8b2681b26ef8", - "resource": { - "resourceType": "Immunization", - "id": "4d5ae690-6eaf-41c2-bee1-8b2681b26ef8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:259c0a73-dc9e-41ee-b821-ce1aadb2faff", - "resource": { - "resourceType": "Immunization", - "id": "259c0a73-dc9e-41ee-b821-ce1aadb2faff", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7b669084-0084-4c89-b97d-59fc357cfa79", - "resource": { - "resourceType": "Immunization", - "id": "7b669084-0084-4c89-b97d-59fc357cfa79", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:72cb0040-8ca0-4c6d-8f30-263006270063", - "resource": { - "resourceType": "Claim", - "id": "72cb0040-8ca0-4c6d-8f30-263006270063", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:43:10-08:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:935d5e93-e972-4d27-8ab7-38afe2a51f91" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:35f80ad5-66d2-49d0-a5fe-68653071b0de" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4d5ae690-6eaf-41c2-bee1-8b2681b26ef8" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:259c0a73-dc9e-41ee-b821-ce1aadb2faff" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7b669084-0084-4c89-b97d-59fc357cfa79" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:ecde24ef-7159-4ad5-9bdf-df56d857f7cb" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:58c3e07d-b23e-4d7c-a85e-9662620a062d" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 306.13, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:94b41851-7237-43ec-a540-ef01cc05503d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "94b41851-7237-43ec-a540-ef01cc05503d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "72cb0040-8ca0-4c6d-8f30-263006270063" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-12-10T05:43:10-08:00", - "end": "2019-12-10T05:43:10-08:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 61.226, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 244.904, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 306.13, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 306.13, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 806.984, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762", - "resource": { - "resourceType": "Encounter", - "id": "7f9b3142-62ee-4763-a46c-f733c123a762", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2019-03-11T06:13:10-07:00", - "end": "2019-03-11T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c8521971-2409-470d-b629-78b2e562907a", - "resource": { - "resourceType": "Observation", - "id": "c8521971-2409-470d-b629-78b2e562907a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 70.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b78488df-4f15-44d7-85f7-9cc577ad6114", - "resource": { - "resourceType": "Observation", - "id": "b78488df-4f15-44d7-85f7-9cc577ad6114", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:224c5de2-616a-4aeb-af70-92849e48bedf", - "resource": { - "resourceType": "Observation", - "id": "224c5de2-616a-4aeb-af70-92849e48bedf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 7.9000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1755540b-e461-4aa3-8994-cd6d2b3af948", - "resource": { - "resourceType": "Observation", - "id": "1755540b-e461-4aa3-8994-cd6d2b3af948", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 7.3337, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f398044e-2bb4-4387-85fe-97397b05cb76", - "resource": { - "resourceType": "Observation", - "id": "f398044e-2bb4-4387-85fe-97397b05cb76", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 44.990, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4f9f6c3-e7f5-4b67-8015-d0519a4a1ace", - "resource": { - "resourceType": "Observation", - "id": "c4f9f6c3-e7f5-4b67-8015-d0519a4a1ace", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 107, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5fbb7b6-2a73-4729-88e1-cf49312d947d", - "resource": { - "resourceType": "Observation", - "id": "a5fbb7b6-2a73-4729-88e1-cf49312d947d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 73, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71c86983-4255-4f27-83f8-c8d74bfbc56e", - "resource": { - "resourceType": "Observation", - "id": "71c86983-4255-4f27-83f8-c8d74bfbc56e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1bbb0806-655b-4936-bf63-e32f3b4d6646", - "resource": { - "resourceType": "Observation", - "id": "1bbb0806-655b-4936-bf63-e32f3b4d6646", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:af277a3c-5cc6-42a9-8548-5425320b7193", - "resource": { - "resourceType": "Procedure", - "id": "af277a3c-5cc6-42a9-8548-5425320b7193", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "performedPeriod": { - "start": "2019-03-11T06:13:10-07:00", - "end": "2019-03-11T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:323b5d1e-e933-46d1-b613-60455686a0b3", - "resource": { - "resourceType": "Immunization", - "id": "323b5d1e-e933-46d1-b613-60455686a0b3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e24b017a-362f-45d4-be05-ab94edf20ac9", - "resource": { - "resourceType": "Immunization", - "id": "e24b017a-362f-45d4-be05-ab94edf20ac9", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c7051b58-38f9-41de-924a-f177c87bfe0c", - "resource": { - "resourceType": "Immunization", - "id": "c7051b58-38f9-41de-924a-f177c87bfe0c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0abe138d-5c05-46a9-8922-15fa27b0ae4e", - "resource": { - "resourceType": "Immunization", - "id": "0abe138d-5c05-46a9-8922-15fa27b0ae4e", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:496449ca-cd40-4cbc-a07b-ebdb28832c92", - "resource": { - "resourceType": "Immunization", - "id": "496449ca-cd40-4cbc-a07b-ebdb28832c92", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b007e486-5c0a-402e-a7da-e7234215dd64", - "resource": { - "resourceType": "Claim", - "id": "b007e486-5c0a-402e-a7da-e7234215dd64", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-03-11T06:13:10-07:00", - "end": "2019-03-11T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:323b5d1e-e933-46d1-b613-60455686a0b3" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e24b017a-362f-45d4-be05-ab94edf20ac9" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c7051b58-38f9-41de-924a-f177c87bfe0c" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:0abe138d-5c05-46a9-8922-15fa27b0ae4e" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:496449ca-cd40-4cbc-a07b-ebdb28832c92" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:af277a3c-5cc6-42a9-8548-5425320b7193" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:7f9b3142-62ee-4763-a46c-f733c123a762" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 455.84, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:903d5a6b-46f2-4886-b290-642c13d1a2c8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "903d5a6b-46f2-4886-b290-642c13d1a2c8", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b007e486-5c0a-402e-a7da-e7234215dd64" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-03-11T06:43:10-07:00", - "end": "2020-03-11T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 91.168, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 364.672, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 455.84, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 455.84, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 926.7520000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba", - "resource": { - "resourceType": "Encounter", - "id": "7e1e68c2-c941-411d-9008-1edd35390bba", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-05-31T06:13:10-07:00", - "end": "2019-05-31T06:43:10-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47", - "resource": { - "resourceType": "Condition", - "id": "41b800ee-0377-47d3-a656-7cf5c49f8a47", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" - }, - "onsetDateTime": "2019-05-31T06:13:10-07:00", - "abatementDateTime": "2019-06-09T06:13:10-07:00", - "assertedDate": "2019-05-31T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f054dc09-7aee-485e-a07e-a49f45b70a11", - "resource": { - "resourceType": "Observation", - "id": "f054dc09-7aee-485e-a07e-a49f45b70a11", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" - }, - "effectiveDateTime": "2019-05-31T06:13:10-07:00", - "issued": "2019-05-31T06:13:10.107-07:00", - "valueQuantity": { - "value": 37.204, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:86ae819c-bff4-428a-a62d-ce767110ea6d", - "resource": { - "resourceType": "Procedure", - "id": "86ae819c-bff4-428a-a62d-ce767110ea6d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" - }, - "performedPeriod": { - "start": "2019-05-31T06:13:10-07:00", - "end": "2019-05-31T06:28:10-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47", - "display": "Acute viral pharyngitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b7c9497e-c1db-4f15-bdf2-2cf78163b154", - "resource": { - "resourceType": "Claim", - "id": "b7c9497e-c1db-4f15-bdf2-2cf78163b154", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-05-31T06:13:10-07:00", - "end": "2019-05-31T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:86ae819c-bff4-428a-a62d-ce767110ea6d" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:7e1e68c2-c941-411d-9008-1edd35390bba" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "net": { - "value": 2006.36, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cd691ae2-9136-4be9-9a88-6b436f5f1b10", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cd691ae2-9136-4be9-9a88-6b436f5f1b10", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b7c9497e-c1db-4f15-bdf2-2cf78163b154" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-05-31T06:43:10-07:00", - "end": "2020-05-31T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:41b800ee-0377-47d3-a656-7cf5c49f8a47" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 401.272, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1605.088, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2006.36, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2006.36, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1605.088, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843", - "resource": { - "resourceType": "Encounter", - "id": "b7b6f2ee-41e5-4010-912d-f63e6f2b7843", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2019-06-10T06:13:10-07:00", - "end": "2019-06-10T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d4690ecb-a5e4-472b-a773-26ffef7cd6d6", - "resource": { - "resourceType": "Observation", - "id": "d4690ecb-a5e4-472b-a773-26ffef7cd6d6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 74.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:764ceafa-2dff-4361-8a8d-95b3e3bc59cd", - "resource": { - "resourceType": "Observation", - "id": "764ceafa-2dff-4361-8a8d-95b3e3bc59cd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:741be9b9-87cf-43c7-a069-4e0a4dd06651", - "resource": { - "resourceType": "Observation", - "id": "741be9b9-87cf-43c7-a069-4e0a4dd06651", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 8.8000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf87b938-858c-42d3-835b-8d28c90b7717", - "resource": { - "resourceType": "Observation", - "id": "cf87b938-858c-42d3-835b-8d28c90b7717", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 8.3146, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66ff3c55-6b91-4695-8219-43af239d6711", - "resource": { - "resourceType": "Observation", - "id": "66ff3c55-6b91-4695-8219-43af239d6711", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 46.130, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:64c391ce-23b9-4257-83e0-cd6133d52079", - "resource": { - "resourceType": "Observation", - "id": "64c391ce-23b9-4257-83e0-cd6133d52079", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 120, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:37d2185b-720b-40bb-a96e-a1c25783dd81", - "resource": { - "resourceType": "Observation", - "id": "37d2185b-720b-40bb-a96e-a1c25783dd81", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:757efdd1-f513-463e-a960-db0cd95c411a", - "resource": { - "resourceType": "Observation", - "id": "757efdd1-f513-463e-a960-db0cd95c411a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b0b3116-c1fd-4a5f-9af9-e543a07391b8", - "resource": { - "resourceType": "Observation", - "id": "4b0b3116-c1fd-4a5f-9af9-e543a07391b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c0103e5-f76a-4d85-b0f0-18afd7c82df1", - "resource": { - "resourceType": "Claim", - "id": "5c0103e5-f76a-4d85-b0f0-18afd7c82df1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-06-10T06:13:10-07:00", - "end": "2019-06-10T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b7b6f2ee-41e5-4010-912d-f63e6f2b7843" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:04f81bb1-b318-4a02-abf8-0203ab4d2992", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "04f81bb1-b318-4a02-abf8-0203ab4d2992", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5c0103e5-f76a-4d85-b0f0-18afd7c82df1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-06-10T06:28:10-07:00", - "end": "2020-06-10T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2", - "resource": { - "resourceType": "Encounter", - "id": "2071123a-e5b2-44e7-b27d-67b03437e6b2", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T07:13:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d", - "resource": { - "resourceType": "Condition", - "id": "9e7ec09d-4469-46e7-b90c-279a291a582d", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "44465007", - "display": "Sprain of ankle" - } - ], - "text": "Sprain of ankle" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - }, - "onsetDateTime": "2019-07-27T06:13:10-07:00", - "abatementDateTime": "2019-08-24T06:13:10-07:00", - "assertedDate": "2019-07-27T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:7aac826d-7e6d-487f-85f5-2ad1d864ffe0", - "resource": { - "resourceType": "MedicationRequest", - "id": "7aac826d-7e6d-487f-85f5-2ad1d864ffe0", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - }, - "authoredOn": "2019-07-27T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1e69cc56-a03f-465a-8cf3-dd662dc91f37", - "resource": { - "resourceType": "Claim", - "id": "1e69cc56-a03f-465a-8cf3-dd662dc91f37", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T07:13:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:7aac826d-7e6d-487f-85f5-2ad1d864ffe0" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - } - ] - } - ], - "total": { - "value": 4.93, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6613c2da-3c5b-4499-a6c4-18a47ff1aa4b", - "resource": { - "resourceType": "CarePlan", - "id": "6613c2da-3c5b-4499-a6c4-18a47ff1aa4b", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "91251008", - "display": "Physical therapy procedure" - } - ], - "text": "Physical therapy procedure" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - }, - "period": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-08-24T06:13:10-07:00" - }, - "addresses": [ - { - "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229586001", - "display": "Rest, ice, compression and elevation treatment programme" - } - ], - "text": "Rest, ice, compression and elevation treatment programme" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "229070002", - "display": "Stretching exercises" - } - ], - "text": "Stretching exercises" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:fbdf870f-6856-4759-b821-d721ee3fe9fd", - "resource": { - "resourceType": "Claim", - "id": "fbdf870f-6856-4759-b821-d721ee3fe9fd", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T07:13:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:27678b31-0de7-492c-9574-e0866c6eea6f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "27678b31-0de7-492c-9574-e0866c6eea6f", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fbdf870f-6856-4759-b821-d721ee3fe9fd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-07-27T07:13:10-07:00", - "end": "2020-07-27T07:13:10-07:00" - }, - "created": "2019-07-27T07:13:10-07:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:fbdf870f-6856-4759-b821-d721ee3fe9fd" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e7ec09d-4469-46e7-b90c-279a291a582d" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T07:13:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:2071123a-e5b2-44e7-b27d-67b03437e6b2" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T07:13:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf", - "resource": { - "resourceType": "Encounter", - "id": "cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f2474743-e217-4fce-9348-ab283d81d698", - "resource": { - "resourceType": "CarePlan", - "id": "f2474743-e217-4fce-9348-ab283d81d698", - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "384758001", - "display": "Self-care interventions (procedure)" - } - ], - "text": "Self-care interventions (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf" - }, - "period": { - "start": "2019-07-27T06:13:10-07:00" - }, - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "409002", - "display": "Food allergy diet" - } - ], - "text": "Food allergy diet" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58332002", - "display": "Allergy education" - } - ], - "text": "Allergy education" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58332002", - "display": "Allergy education" - } - ], - "text": "Allergy education" - }, - "status": "in-progress" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:67bbb47b-b627-411b-a12e-e8b8b4073fa0", - "resource": { - "resourceType": "Claim", - "id": "67bbb47b-b627-411b-a12e-e8b8b4073fa0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-07-27T06:13:10-07:00", - "end": "2019-07-27T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:cc1f78a7-e7b4-4d7c-b06b-a512c2809fbf" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d7509e59-6617-434a-b10c-7a2792dc2031", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d7509e59-6617-434a-b10c-7a2792dc2031", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "67bbb47b-b627-411b-a12e-e8b8b4073fa0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-07-27T06:28:10-07:00", - "end": "2020-07-27T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862", - "resource": { - "resourceType": "Encounter", - "id": "6427742c-bd2c-4f4c-aea4-644f5db83862", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-08-11T06:13:10-07:00", - "end": "2019-08-11T07:06:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6db8de76-eae5-4950-8a42-2607004386fd", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "6db8de76-eae5-4950-8a42-2607004386fd", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "232347008", - "display": "Dander (animal) allergy" - } - ], - "text": "Dander (animal) allergy" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "assertedDate": "2019-08-11T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:e35c9f21-9093-4e84-9b36-615ec2d1c210", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "e35c9f21-9093-4e84-9b36-615ec2d1c210", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "300913006", - "display": "Shellfish allergy" - } - ], - "text": "Shellfish allergy" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "assertedDate": "2019-08-11T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:625e1745-69bc-426b-aca1-10e83724f849", - "resource": { - "resourceType": "Observation", - "id": "625e1745-69bc-426b-aca1-10e83724f849", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6206-7", - "display": "Peanut IgE Ab in Serum" - } - ], - "text": "Peanut IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.10165, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ca6457f-b5f8-4203-8ad5-be9b1c140026", - "resource": { - "resourceType": "Observation", - "id": "0ca6457f-b5f8-4203-8ad5-be9b1c140026", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6273-7", - "display": "Walnut IgE Ab in Serum" - } - ], - "text": "Walnut IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.19633, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:de18b9d6-84e1-40d3-ab38-0b97a85537c5", - "resource": { - "resourceType": "Observation", - "id": "de18b9d6-84e1-40d3-ab38-0b97a85537c5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6082-2", - "display": "Codfish IgE Ab in Serum" - } - ], - "text": "Codfish IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.14530, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:07d68b6c-2868-4402-95cc-8ad376c3d8e1", - "resource": { - "resourceType": "Observation", - "id": "07d68b6c-2868-4402-95cc-8ad376c3d8e1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6246-3", - "display": "Shrimp IgE Ab in Serum" - } - ], - "text": "Shrimp IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 12.952, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40530c35-df47-4333-84b4-ee421dd431b8", - "resource": { - "resourceType": "Observation", - "id": "40530c35-df47-4333-84b4-ee421dd431b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6276-0", - "display": "Wheat IgE Ab in Serum" - } - ], - "text": "Wheat IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.34990, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77c14a36-4b3c-49b5-824d-9079efd3e215", - "resource": { - "resourceType": "Observation", - "id": "77c14a36-4b3c-49b5-824d-9079efd3e215", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6106-9", - "display": "Egg white IgE Ab in Serum" - } - ], - "text": "Egg white IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.31980, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5d40e233-c707-4989-94f2-a734f60e878d", - "resource": { - "resourceType": "Observation", - "id": "5d40e233-c707-4989-94f2-a734f60e878d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6248-9", - "display": "Soybean IgE Ab in Serum" - } - ], - "text": "Soybean IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.12029, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:85ca8f32-f13f-4ab6-80dd-a198fdc07d5d", - "resource": { - "resourceType": "Observation", - "id": "85ca8f32-f13f-4ab6-80dd-a198fdc07d5d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "7258-7", - "display": "Cow milk IgE Ab in Serum" - } - ], - "text": "Cow milk IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.34789, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c421021b-5667-40ec-af43-0f77d7429d24", - "resource": { - "resourceType": "Observation", - "id": "c421021b-5667-40ec-af43-0f77d7429d24", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6189-5", - "display": "White oak IgE Ab in Serum" - } - ], - "text": "White oak IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.27479, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40ae2b98-265a-4626-9b5e-b3cbaf728fa4", - "resource": { - "resourceType": "Observation", - "id": "40ae2b98-265a-4626-9b5e-b3cbaf728fa4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6085-5", - "display": "Common Ragweed IgE Ab in Serum" - } - ], - "text": "Common Ragweed IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.18551, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a87c4cc7-bda8-4c88-a570-0fc40e10fdf9", - "resource": { - "resourceType": "Observation", - "id": "a87c4cc7-bda8-4c88-a570-0fc40e10fdf9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6833-8", - "display": "Cat dander IgE Ab in Serum" - } - ], - "text": "Cat dander IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 24.424, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a45da77-5881-4753-b85e-836bd3161449", - "resource": { - "resourceType": "Observation", - "id": "3a45da77-5881-4753-b85e-836bd3161449", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6095-4", - "display": "American house dust mite IgE Ab in Serum" - } - ], - "text": "American house dust mite IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.20303, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:688791a4-03f4-45ab-8462-2c41b22cfc3c", - "resource": { - "resourceType": "Observation", - "id": "688791a4-03f4-45ab-8462-2c41b22cfc3c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6075-6", - "display": "Cladosporium herbarum IgE Ab in Serum" - } - ], - "text": "Cladosporium herbarum IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.23219, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5de5308-150c-4eef-a9de-d76e06f02656", - "resource": { - "resourceType": "Observation", - "id": "f5de5308-150c-4eef-a9de-d76e06f02656", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6844-5", - "display": "Honey bee IgE Ab in Serum" - } - ], - "text": "Honey bee IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.13334, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f898ef50-0e14-4344-8c54-57ca6e5208c7", - "resource": { - "resourceType": "Observation", - "id": "f898ef50-0e14-4344-8c54-57ca6e5208c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6158-0", - "display": "Latex IgE Ab in Serum" - } - ], - "text": "Latex IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "effectiveDateTime": "2019-08-11T06:13:10-07:00", - "issued": "2019-08-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.31837, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e309a89d-4eef-4383-b200-901f2b6a60e5", - "resource": { - "resourceType": "Procedure", - "id": "e309a89d-4eef-4383-b200-901f2b6a60e5", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "395142003", - "display": "Allergy screening test" - } - ], - "text": "Allergy screening test" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "performedPeriod": { - "start": "2019-08-11T06:13:10-07:00", - "end": "2019-08-11T06:51:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e2256164-5e0e-49fe-bd55-ae3b268b7901", - "resource": { - "resourceType": "MedicationRequest", - "id": "e2256164-5e0e-49fe-bd55-ae3b268b7901", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1014676", - "display": "cetirizine hydrochloride 5 MG Oral Tablet" - } - ], - "text": "cetirizine hydrochloride 5 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "authoredOn": "2019-08-11T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:75b5cd63-28a6-468e-9fdd-ebbdbe88987e", - "resource": { - "resourceType": "Claim", - "id": "75b5cd63-28a6-468e-9fdd-ebbdbe88987e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-08-11T06:13:10-07:00", - "end": "2019-08-11T07:06:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:e2256164-5e0e-49fe-bd55-ae3b268b7901" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - } - ] - } - ], - "total": { - "value": 20.99, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b5db7fae-fb16-47e5-8eb4-39ace8c18c17", - "resource": { - "resourceType": "MedicationRequest", - "id": "b5db7fae-fb16-47e5-8eb4-39ace8c18c17", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1870230", - "display": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" - } - ], - "text": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - }, - "authoredOn": "2019-08-11T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8b72f548-4002-453b-b0bc-aa8caeda7b49", - "resource": { - "resourceType": "Claim", - "id": "8b72f548-4002-453b-b0bc-aa8caeda7b49", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-08-11T06:13:10-07:00", - "end": "2019-08-11T07:06:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:b5db7fae-fb16-47e5-8eb4-39ace8c18c17" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - } - ] - } - ], - "total": { - "value": 258.81, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8b74a937-8421-4f63-b9a2-423e182e62b3", - "resource": { - "resourceType": "Claim", - "id": "8b74a937-8421-4f63-b9a2-423e182e62b3", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-08-11T06:13:10-07:00", - "end": "2019-08-11T07:06:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e309a89d-4eef-4383-b200-901f2b6a60e5" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:6427742c-bd2c-4f4c-aea4-644f5db83862" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "395142003" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:da38110f-ae59-40ac-ad2b-bc65ba12bf11", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "da38110f-ae59-40ac-ad2b-bc65ba12bf11", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8b74a937-8421-4f63-b9a2-423e182e62b3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-08-11T07:06:10-07:00", - "end": "2020-08-11T07:06:10-07:00" - }, - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "395142003" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c", - "resource": { - "resourceType": "Encounter", - "id": "5aa23372-8c55-4ba4-80c6-37be3d22949c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2019-09-09T06:13:10-07:00", - "end": "2019-09-09T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:78f74aa9-e375-4759-aeb3-dd0b6559b3af", - "resource": { - "resourceType": "Observation", - "id": "78f74aa9-e375-4759-aeb3-dd0b6559b3af", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 78.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8233134-6384-4f59-90f8-bcb62762305d", - "resource": { - "resourceType": "Observation", - "id": "d8233134-6384-4f59-90f8-bcb62762305d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:891ee358-94ad-4f1c-8fb5-0277b633b591", - "resource": { - "resourceType": "Observation", - "id": "891ee358-94ad-4f1c-8fb5-0277b633b591", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 9.6000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59dc9644-adbc-44e4-9f03-deef443a1b98", - "resource": { - "resourceType": "Observation", - "id": "59dc9644-adbc-44e4-9f03-deef443a1b98", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 12.951, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ceeba545-7627-4585-b24b-3cd96903c26b", - "resource": { - "resourceType": "Observation", - "id": "ceeba545-7627-4585-b24b-3cd96903c26b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 46.970, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2e1cd59f-a823-404e-beb6-c80fcda809a3", - "resource": { - "resourceType": "Observation", - "id": "2e1cd59f-a823-404e-beb6-c80fcda809a3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:be2ffbce-f21b-4abc-983e-7b91f81d6390", - "resource": { - "resourceType": "Observation", - "id": "be2ffbce-f21b-4abc-983e-7b91f81d6390", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 72, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb29d6dd-badb-43ad-abed-4d18b39a4313", - "resource": { - "resourceType": "Observation", - "id": "fb29d6dd-badb-43ad-abed-4d18b39a4313", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d27a850-b426-466d-b8f8-5392dee55185", - "resource": { - "resourceType": "Observation", - "id": "3d27a850-b426-466d-b8f8-5392dee55185", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0163a714-c267-4559-ba24-4374d2f70bf3", - "resource": { - "resourceType": "Immunization", - "id": "0163a714-c267-4559-ba24-4374d2f70bf3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d2d20c61-a742-4d8f-b26e-da5e5b862cc5", - "resource": { - "resourceType": "Immunization", - "id": "d2d20c61-a742-4d8f-b26e-da5e5b862cc5", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2b9227b0-2535-4c7b-977c-6f9899ef53c2", - "resource": { - "resourceType": "Immunization", - "id": "2b9227b0-2535-4c7b-977c-6f9899ef53c2", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c57c49d3-5f3a-47e3-808f-fb6f315dea40", - "resource": { - "resourceType": "Immunization", - "id": "c57c49d3-5f3a-47e3-808f-fb6f315dea40", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:198993c4-73fd-4314-909d-3a0d9770434b", - "resource": { - "resourceType": "Immunization", - "id": "198993c4-73fd-4314-909d-3a0d9770434b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77", - "resource": { - "resourceType": "Claim", - "id": "77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-09-09T06:13:10-07:00", - "end": "2019-09-09T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:0163a714-c267-4559-ba24-4374d2f70bf3" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d2d20c61-a742-4d8f-b26e-da5e5b862cc5" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:2b9227b0-2535-4c7b-977c-6f9899ef53c2" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c57c49d3-5f3a-47e3-808f-fb6f315dea40" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:198993c4-73fd-4314-909d-3a0d9770434b" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5aa23372-8c55-4ba4-80c6-37be3d22949c" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3b0dd334-0e8a-4f21-8d88-c5c74168dd9b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3b0dd334-0e8a-4f21-8d88-c5c74168dd9b", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "77a06eb1-0fc6-4c9f-b5bc-30dcdfa69a77" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-09-09T06:28:10-07:00", - "end": "2020-09-09T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897", - "resource": { - "resourceType": "Encounter", - "id": "4a0a3706-4dec-4bd1-bcd2-04b43b9b5897", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:43:10-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e52b282f-d0db-4563-9fa6-b4c4fd9b2634", - "resource": { - "resourceType": "Observation", - "id": "e52b282f-d0db-4563-9fa6-b4c4fd9b2634", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 81.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:345104fe-7d72-401e-b86e-6e2c46332075", - "resource": { - "resourceType": "Observation", - "id": "345104fe-7d72-401e-b86e-6e2c46332075", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:144df2bb-df62-41db-95d4-bc817c50c768", - "resource": { - "resourceType": "Observation", - "id": "144df2bb-df62-41db-95d4-bc817c50c768", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 10.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:295341fa-d93c-43e6-bc77-020682f7da79", - "resource": { - "resourceType": "Observation", - "id": "295341fa-d93c-43e6-bc77-020682f7da79", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 9.5225, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e7c51d7b-3207-40a8-b286-d6c165c708fb", - "resource": { - "resourceType": "Observation", - "id": "e7c51d7b-3207-40a8-b286-d6c165c708fb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 47.610, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a73cf61b-f65d-4a99-9ba2-b30eb1ca230b", - "resource": { - "resourceType": "Observation", - "id": "a73cf61b-f65d-4a99-9ba2-b30eb1ca230b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2286c5f5-9bf5-4355-882b-e0639555fe49", - "resource": { - "resourceType": "Observation", - "id": "2286c5f5-9bf5-4355-882b-e0639555fe49", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9af05c49-221f-4c55-9551-cd580f69253d", - "resource": { - "resourceType": "Observation", - "id": "9af05c49-221f-4c55-9551-cd580f69253d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b19b739b-9393-4360-b700-7c6dc0c012c1", - "resource": { - "resourceType": "Observation", - "id": "b19b739b-9393-4360-b700-7c6dc0c012c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7adda0ad-c08a-4436-b059-c0d91b7fe86d", - "resource": { - "resourceType": "Procedure", - "id": "7adda0ad-c08a-4436-b059-c0d91b7fe86d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "performedPeriod": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:28:10-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:8fc177fe-6983-491a-abb3-c594e8095837", - "resource": { - "resourceType": "Immunization", - "id": "8fc177fe-6983-491a-abb3-c594e8095837", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "encounter": { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - }, - "date": "2019-12-09T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c6be629b-8271-42ec-a56d-a4e5318bca81", - "resource": { - "resourceType": "Claim", - "id": "c6be629b-8271-42ec-a56d-a4e5318bca81", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:43:10-08:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8fc177fe-6983-491a-abb3-c594e8095837" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7adda0ad-c08a-4436-b059-c0d91b7fe86d" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4a0a3706-4dec-4bd1-bcd2-04b43b9b5897" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 671.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:61596942-9e73-4464-9cd7-893f73c722b8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "61596942-9e73-4464-9cd7-893f73c722b8", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c6be629b-8271-42ec-a56d-a4e5318bca81" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-12-09T05:43:10-08:00", - "end": "2020-12-09T05:43:10-08:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 134.346, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 537.384, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 671.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 671.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 649.8000000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3", - "resource": { - "resourceType": "Encounter", - "id": "b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308646001", - "display": "Death Certification" - } - ], - "text": "Death Certification" - } - ], - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886" - } - } - ], - "period": { - "start": "2020-04-13T06:13:10-07:00", - "end": "2020-04-13T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4a076392-7755-4007-bd4e-516b124cceb5", - "resource": { - "resourceType": "Observation", - "id": "4a076392-7755-4007-bd4e-516b124cceb5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "exam", - "display": "exam" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "69453-9", - "display": "Cause of Death [US Standard Certificate of Death]" - } - ], - "text": "Cause of Death [US Standard Certificate of Death]" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" - }, - "effectiveDateTime": "2020-04-13T06:13:10-07:00", - "issued": "2020-04-13T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "95281009", - "display": "Sudden Cardiac Death" - } - ], - "text": "Sudden Cardiac Death" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cef6a500-0ea0-41fe-8606-bf3221180ba7", - "resource": { - "resourceType": "DiagnosticReport", - "id": "cef6a500-0ea0-41fe-8606-bf3221180ba7", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "69409-1", - "display": "U.S. standard certificate of death - 2003 revision" - } - ], - "text": "U.S. standard certificate of death - 2003 revision" - }, - "subject": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "context": { - "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" - }, - "effectiveDateTime": "2020-04-13T06:13:10-07:00", - "issued": "2020-04-13T06:13:10.107-07:00", - "result": [ - { - "reference": "urn:uuid:4a076392-7755-4007-bd4e-516b124cceb5", - "display": "Cause of Death [US Standard Certificate of Death]" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:100daf7c-469d-40e7-b263-184f2f44d223", - "resource": { - "resourceType": "Claim", - "id": "100daf7c-469d-40e7-b263-184f2f44d223", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "start": "2020-04-13T06:13:10-07:00", - "end": "2020-04-13T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b1110ba5-b66d-48bd-98ca-b2cc28ad6eb3" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0c772652-8ab7-4e11-a049-a86d04c2bbb5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0c772652-8ab7-4e11-a049-a86d04c2bbb5", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "100daf7c-469d-40e7-b263-184f2f44d223" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:02a63c07-9fcc-42ba-aec0-9d5399ac4796" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2020-04-13T06:28:10-07:00", - "end": "2021-04-13T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "29e6e246-9052-309c-a47f-c2e5bc60c886" - } - }, - "organization": { - "identifier": { - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json deleted file mode 100644 index 1bf7f49aa84b..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Basil991_Pfannerstill264_88fb71f7-d445-4e5b-8af2-962e8f8e5fb6.json +++ /dev/null @@ -1,15372 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6", - "resource": { - "resourceType": "Patient", - "id": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -3160153991630561462 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2054-5", - "display": "Black or African American" - } - }, - { - "url": "text", - "valueString": "Black or African American" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Asia282 Crona259" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "M" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Wellesley", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.0 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 1.0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-12-5406" - } - ], - "name": [ - { - "use": "official", - "family": "Pfannerstill264", - "given": [ - "Basil991" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-373-9418", - "use": "home" - } - ], - "gender": "male", - "birthDate": "2018-07-02", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.40406475290831 - }, - { - "url": "longitude", - "valueDecimal": -71.12753835800636 - } - ] - } - ], - "line": [ - "720 Hickle Light Apt 14" - ], - "city": "Cambridge", - "state": "Massachusetts", - "postalCode": "02472", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212", - "resource": { - "resourceType": "Organization", - "id": "83284578-12e5-3582-bfe1-7807e3f7a212", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP297713", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.423844 - }, - { - "url": "longitude", - "valueDecimal": -71.10923100000001 - } - ] - } - ], - "line": [ - "0 GOVERNORS AVE" - ], - "city": "MEDFORD", - "state": "MA", - "postalCode": "02155-3084", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499", - "resource": { - "resourceType": "Practitioner", - "id": "d726e757-2975-3d6a-bdf9-517546432499", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "80690" - } - ], - "active": true, - "name": [ - { - "family": "Abernathy524", - "given": [ - "Ulrike696" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "0 GOVERNORS AVE" - ], - "city": "MEDFORD", - "state": "MA", - "postalCode": "02155-3084", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a", - "resource": { - "resourceType": "Encounter", - "id": "41d9c565-feb8-49b7-8dc8-c4900f98793a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3f83f9ad-288b-43a2-a4f2-7c6fb421f525", - "resource": { - "resourceType": "Observation", - "id": "3f83f9ad-288b-43a2-a4f2-7c6fb421f525", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 50.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51654efa-2ee3-4acb-9788-12a8da445dc3", - "resource": { - "resourceType": "Observation", - "id": "51654efa-2ee3-4acb-9788-12a8da445dc3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6015ade7-061d-4fcd-bbf9-0e3c7b5342f1", - "resource": { - "resourceType": "Observation", - "id": "6015ade7-061d-4fcd-bbf9-0e3c7b5342f1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 3.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9357cf13-166e-4e60-a73f-6be78861709d", - "resource": { - "resourceType": "Observation", - "id": "9357cf13-166e-4e60-a73f-6be78861709d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 41.793, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8c89cd9d-b297-474e-bb30-9b79917684c8", - "resource": { - "resourceType": "Observation", - "id": "8c89cd9d-b297-474e-bb30-9b79917684c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 33.670, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9603a9c5-a600-472d-aaa4-2c618566a50e", - "resource": { - "resourceType": "Observation", - "id": "9603a9c5-a600-472d-aaa4-2c618566a50e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 104, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:59171c7d-b931-44f0-b0aa-117a6520d463", - "resource": { - "resourceType": "Observation", - "id": "59171c7d-b931-44f0-b0aa-117a6520d463", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 84, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:483fa61e-36df-4205-ad60-dbecca018ff5", - "resource": { - "resourceType": "Observation", - "id": "483fa61e-36df-4205-ad60-dbecca018ff5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:305ca67d-1925-472b-b350-a8997785402d", - "resource": { - "resourceType": "Observation", - "id": "305ca67d-1925-472b-b350-a8997785402d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 8.0387, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b48ddff0-b112-4074-a71e-9fa48369ea8f", - "resource": { - "resourceType": "Observation", - "id": "b48ddff0-b112-4074-a71e-9fa48369ea8f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 4.2691, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1428d255-36bc-4c2c-bfd0-07b812870867", - "resource": { - "resourceType": "Observation", - "id": "1428d255-36bc-4c2c-bfd0-07b812870867", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 13.901, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:afff4d3d-b6ca-40bd-8eee-e72996840ead", - "resource": { - "resourceType": "Observation", - "id": "afff4d3d-b6ca-40bd-8eee-e72996840ead", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 40.693, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b4e5944-6741-434c-b419-4a428d31919f", - "resource": { - "resourceType": "Observation", - "id": "2b4e5944-6741-434c-b419-4a428d31919f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 80.501, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", - "resource": { - "resourceType": "Observation", - "id": "9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 30.643, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:64801d0a-09b0-41fc-90bc-5ec827640cca", - "resource": { - "resourceType": "Observation", - "id": "64801d0a-09b0-41fc-90bc-5ec827640cca", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 33.848, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:22f92c59-c8cc-4d52-9ada-53a218396179", - "resource": { - "resourceType": "Observation", - "id": "22f92c59-c8cc-4d52-9ada-53a218396179", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 42.600, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", - "resource": { - "resourceType": "Observation", - "id": "2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 420.43, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:350edd3f-f32a-4b5c-b05a-f1e26b944f8a", - "resource": { - "resourceType": "Observation", - "id": "350edd3f-f32a-4b5c-b05a-f1e26b944f8a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 233.63, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12d5bae9-94a3-458a-a398-b49bfa340628", - "resource": { - "resourceType": "Observation", - "id": "12d5bae9-94a3-458a-a398-b49bfa340628", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueQuantity": { - "value": 9.4122, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5e895c0-210b-43bb-b2cb-e7c539768cf8", - "resource": { - "resourceType": "Observation", - "id": "f5e895c0-210b-43bb-b2cb-e7c539768cf8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a28f96b8-1a93-473a-9454-dccb88955db7", - "resource": { - "resourceType": "Procedure", - "id": "a28f96b8-1a93-473a-9454-dccb88955db7", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "performedPeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c27f8d19-48e1-4739-89fc-141e1a9d7ce6", - "resource": { - "resourceType": "Immunization", - "id": "c27f8d19-48e1-4739-89fc-141e1a9d7ce6", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "date": "2018-07-02T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a0fac80f-483e-41f2-b361-ca919e400c71", - "resource": { - "resourceType": "DiagnosticReport", - "id": "a0fac80f-483e-41f2-b361-ca919e400c71", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - }, - "effectiveDateTime": "2018-07-02T06:13:10-07:00", - "issued": "2018-07-02T06:13:10.107-07:00", - "result": [ - { - "reference": "urn:uuid:305ca67d-1925-472b-b350-a8997785402d", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b48ddff0-b112-4074-a71e-9fa48369ea8f", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1428d255-36bc-4c2c-bfd0-07b812870867", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:afff4d3d-b6ca-40bd-8eee-e72996840ead", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:2b4e5944-6741-434c-b419-4a428d31919f", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:9be29bc7-57ad-4a20-a7ec-c48b379ea3bb", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:64801d0a-09b0-41fc-90bc-5ec827640cca", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:22f92c59-c8cc-4d52-9ada-53a218396179", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:2bf48a7f-80b1-4ed0-8422-b2e344d8e7df", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:350edd3f-f32a-4b5c-b05a-f1e26b944f8a", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:12d5bae9-94a3-458a-a398-b49bfa340628", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:1a73022b-ab53-496e-8ec4-fd11d353a97a", - "resource": { - "resourceType": "Claim", - "id": "1a73022b-ab53-496e-8ec4-fd11d353a97a", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2018-07-02T06:13:10-07:00", - "end": "2018-07-02T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c27f8d19-48e1-4739-89fc-141e1a9d7ce6" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a28f96b8-1a93-473a-9454-dccb88955db7" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:41d9c565-feb8-49b7-8dc8-c4900f98793a" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 510.48, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:664de9a4-da89-4727-924a-f6ab9ff52686", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "664de9a4-da89-4727-924a-f6ab9ff52686", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1a73022b-ab53-496e-8ec4-fd11d353a97a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-07-02T06:43:10-07:00", - "end": "2019-07-02T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 102.096, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 408.384, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 510.48, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 510.48, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 520.8000000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066", - "resource": { - "resourceType": "Encounter", - "id": "2b2dc0de-65d8-4673-9e28-ee5c85a3d066", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2018-08-06T06:13:10-07:00", - "end": "2018-08-06T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c423e214-b4f6-4601-bf66-729a07205e48", - "resource": { - "resourceType": "Observation", - "id": "c423e214-b4f6-4601-bf66-729a07205e48", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 54.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:acd9d21d-c3f6-4704-a1e0-892946ddc950", - "resource": { - "resourceType": "Observation", - "id": "acd9d21d-c3f6-4704-a1e0-892946ddc950", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9017713d-30ab-4825-be7a-448339eeaf05", - "resource": { - "resourceType": "Observation", - "id": "9017713d-30ab-4825-be7a-448339eeaf05", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 4.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fe916404-d864-4d5b-a2be-9aca37e28687", - "resource": { - "resourceType": "Observation", - "id": "fe916404-d864-4d5b-a2be-9aca37e28687", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 25.763, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf8939f8-e35d-4977-8464-567af8adc3c2", - "resource": { - "resourceType": "Observation", - "id": "cf8939f8-e35d-4977-8464-567af8adc3c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 37.430, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:146779d7-66ba-4005-b989-5c2a5365d422", - "resource": { - "resourceType": "Observation", - "id": "146779d7-66ba-4005-b989-5c2a5365d422", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 70, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 134, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3a5215d7-fcef-45e3-9d25-91d5a8d13df5", - "resource": { - "resourceType": "Observation", - "id": "3a5215d7-fcef-45e3-9d25-91d5a8d13df5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 84, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1dad280f-c7fe-40d0-9bbe-13fa34b9743c", - "resource": { - "resourceType": "Observation", - "id": "1dad280f-c7fe-40d0-9bbe-13fa34b9743c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1604fe32-413e-4571-b1a1-555ad84b9cdf", - "resource": { - "resourceType": "Observation", - "id": "1604fe32-413e-4571-b1a1-555ad84b9cdf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "effectiveDateTime": "2018-08-06T06:13:10-07:00", - "issued": "2018-08-06T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:362ab788-f630-4514-9efa-433dee3edf2e", - "resource": { - "resourceType": "Immunization", - "id": "362ab788-f630-4514-9efa-433dee3edf2e", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - }, - "date": "2018-08-06T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d55e5479-6f83-4d8d-969a-63e18cefeede", - "resource": { - "resourceType": "Claim", - "id": "d55e5479-6f83-4d8d-969a-63e18cefeede", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2018-08-06T06:13:10-07:00", - "end": "2018-08-06T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:362ab788-f630-4514-9efa-433dee3edf2e" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2b2dc0de-65d8-4673-9e28-ee5c85a3d066" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:30538e24-8919-458b-9241-b49073124182", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "30538e24-8919-458b-9241-b49073124182", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d55e5479-6f83-4d8d-969a-63e18cefeede" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-08-06T06:28:10-07:00", - "end": "2019-08-06T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1", - "resource": { - "resourceType": "Encounter", - "id": "e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2018-10-08T06:13:10-07:00", - "end": "2018-10-08T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:be531e9a-d7db-4ec9-81d8-e827032c3752", - "resource": { - "resourceType": "Observation", - "id": "be531e9a-d7db-4ec9-81d8-e827032c3752", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 59.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4569557d-7c4e-4ece-beef-a3909ed9587c", - "resource": { - "resourceType": "Observation", - "id": "4569557d-7c4e-4ece-beef-a3909ed9587c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1261b413-92b1-45de-b4b9-d450c566fe7a", - "resource": { - "resourceType": "Observation", - "id": "1261b413-92b1-45de-b4b9-d450c566fe7a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 5.7000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:980ef229-efdf-417e-9bd3-f7700a6bb531", - "resource": { - "resourceType": "Observation", - "id": "980ef229-efdf-417e-9bd3-f7700a6bb531", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 25.701, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f27810b-aa48-45de-8346-52a7f456aa7e", - "resource": { - "resourceType": "Observation", - "id": "1f27810b-aa48-45de-8346-52a7f456aa7e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 40.230, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:471e0d21-a483-40dc-97d7-3df908d031aa", - "resource": { - "resourceType": "Observation", - "id": "471e0d21-a483-40dc-97d7-3df908d031aa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fbafe674-f215-4f9d-a639-6208e0abf44f", - "resource": { - "resourceType": "Observation", - "id": "fbafe674-f215-4f9d-a639-6208e0abf44f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 83, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36ff5081-58eb-42e6-a119-01712beb9f68", - "resource": { - "resourceType": "Observation", - "id": "36ff5081-58eb-42e6-a119-01712beb9f68", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d75171f-8af8-447f-9bc5-5b2a12e7d7dc", - "resource": { - "resourceType": "Observation", - "id": "9d75171f-8af8-447f-9bc5-5b2a12e7d7dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "effectiveDateTime": "2018-10-08T06:13:10-07:00", - "issued": "2018-10-08T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5b63d12-9e5c-47e5-a887-0a01c14ca57a", - "resource": { - "resourceType": "Procedure", - "id": "a5b63d12-9e5c-47e5-a887-0a01c14ca57a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "performedPeriod": { - "start": "2018-10-08T06:13:10-07:00", - "end": "2018-10-08T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:628d5705-ccb9-43e8-970a-4566b7a0e319", - "resource": { - "resourceType": "Immunization", - "id": "628d5705-ccb9-43e8-970a-4566b7a0e319", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7e1a73c9-11eb-4b4f-979f-9b1382a47db3", - "resource": { - "resourceType": "Immunization", - "id": "7e1a73c9-11eb-4b4f-979f-9b1382a47db3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d3b80ab6-a01d-418e-aba5-11199e756a66", - "resource": { - "resourceType": "Immunization", - "id": "d3b80ab6-a01d-418e-aba5-11199e756a66", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:304b49f5-521e-4f6a-b9cc-9c74ce0e3292", - "resource": { - "resourceType": "Immunization", - "id": "304b49f5-521e-4f6a-b9cc-9c74ce0e3292", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:173ca7d0-b5f4-43d1-8efa-3826118fed63", - "resource": { - "resourceType": "Immunization", - "id": "173ca7d0-b5f4-43d1-8efa-3826118fed63", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - }, - "date": "2018-10-08T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2ee27832-be87-4039-8d48-413bdf2c02db", - "resource": { - "resourceType": "Claim", - "id": "2ee27832-be87-4039-8d48-413bdf2c02db", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2018-10-08T06:13:10-07:00", - "end": "2018-10-08T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:628d5705-ccb9-43e8-970a-4566b7a0e319" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7e1a73c9-11eb-4b4f-979f-9b1382a47db3" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d3b80ab6-a01d-418e-aba5-11199e756a66" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:304b49f5-521e-4f6a-b9cc-9c74ce0e3292" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:173ca7d0-b5f4-43d1-8efa-3826118fed63" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a5b63d12-9e5c-47e5-a887-0a01c14ca57a" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e5718a30-a6cb-4d3b-8973-1dd4c9ae3fb1" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 575.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:624d52c3-267c-479c-9970-bf95e97c525d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "624d52c3-267c-479c-9970-bf95e97c525d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2ee27832-be87-4039-8d48-413bdf2c02db" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-10-08T06:43:10-07:00", - "end": "2019-10-08T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 115.14000000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 460.56000000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 575.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 575.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1022.6400000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f", - "resource": { - "resourceType": "Encounter", - "id": "eefc8fce-bc6b-41a3-b6d1-f731bb560f0f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:43:10-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:05a40880-1366-40af-95c1-5966694a5bdc", - "resource": { - "resourceType": "Observation", - "id": "05a40880-1366-40af-95c1-5966694a5bdc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 63.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:62c1a0fc-fac2-4af0-acc4-35f9fda22498", - "resource": { - "resourceType": "Observation", - "id": "62c1a0fc-fac2-4af0-acc4-35f9fda22498", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a137e5d6-3956-4436-877d-143c29966fb1", - "resource": { - "resourceType": "Observation", - "id": "a137e5d6-3956-4436-877d-143c29966fb1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 6.8000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:461ae86e-efd4-46cf-8f95-9b97b129edde", - "resource": { - "resourceType": "Observation", - "id": "461ae86e-efd4-46cf-8f95-9b97b129edde", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 32.712, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0f029843-14f7-4678-8766-66bd96bdaf5f", - "resource": { - "resourceType": "Observation", - "id": "0f029843-14f7-4678-8766-66bd96bdaf5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 41.990, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:01ac0323-eab5-406a-ad6c-8dfaa12d5b72", - "resource": { - "resourceType": "Observation", - "id": "01ac0323-eab5-406a-ad6c-8dfaa12d5b72", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 102, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e1a89a17-3a00-4783-8a5f-053a052fadaa", - "resource": { - "resourceType": "Observation", - "id": "e1a89a17-3a00-4783-8a5f-053a052fadaa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:38264da7-a579-46d7-aee1-6a44c423e75d", - "resource": { - "resourceType": "Observation", - "id": "38264da7-a579-46d7-aee1-6a44c423e75d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:56023140-1737-445a-b834-49a9b0a932a6", - "resource": { - "resourceType": "Observation", - "id": "56023140-1737-445a-b834-49a9b0a932a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "effectiveDateTime": "2018-12-10T05:13:10-08:00", - "issued": "2018-12-10T05:13:10.107-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2b3f5fd8-7a23-4a20-a571-7287f33e8a12", - "resource": { - "resourceType": "Procedure", - "id": "2b3f5fd8-7a23-4a20-a571-7287f33e8a12", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "performedPeriod": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:28:10-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec", - "resource": { - "resourceType": "Immunization", - "id": "9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f85344ff-d6ed-4c2f-b671-3755d4b0c614", - "resource": { - "resourceType": "Immunization", - "id": "f85344ff-d6ed-4c2f-b671-3755d4b0c614", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1dba5b1b-ef05-4742-b4d8-3c780bcd2439", - "resource": { - "resourceType": "Immunization", - "id": "1dba5b1b-ef05-4742-b4d8-3c780bcd2439", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b93bf8c9-925d-4bfa-838c-25240ef7ff61", - "resource": { - "resourceType": "Immunization", - "id": "b93bf8c9-925d-4bfa-838c-25240ef7ff61", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:33c13591-1d77-42a1-b519-5ecbea921a2e", - "resource": { - "resourceType": "Immunization", - "id": "33c13591-1d77-42a1-b519-5ecbea921a2e", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - }, - "date": "2018-12-10T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:fca0a350-61ec-49d4-8bbd-cd951e53570b", - "resource": { - "resourceType": "Claim", - "id": "fca0a350-61ec-49d4-8bbd-cd951e53570b", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2018-12-10T05:13:10-08:00", - "end": "2018-12-10T05:43:10-08:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:9fb41cd0-30ce-4d7f-acd8-6349b35fa7ec" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f85344ff-d6ed-4c2f-b671-3755d4b0c614" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1dba5b1b-ef05-4742-b4d8-3c780bcd2439" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:b93bf8c9-925d-4bfa-838c-25240ef7ff61" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:33c13591-1d77-42a1-b519-5ecbea921a2e" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:2b3f5fd8-7a23-4a20-a571-7287f33e8a12" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:eefc8fce-bc6b-41a3-b6d1-f731bb560f0f" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 461.41, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:75de1d08-9852-400a-a42e-108e938c26bb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "75de1d08-9852-400a-a42e-108e938c26bb", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fca0a350-61ec-49d4-8bbd-cd951e53570b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-12-10T05:43:10-08:00", - "end": "2019-12-10T05:43:10-08:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 92.28200000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 369.12800000000004, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 461.41, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 461.41, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 931.2080000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578", - "resource": { - "resourceType": "Encounter", - "id": "cd9fa994-4899-4e33-bcbd-7e526b471578", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2019-03-11T06:13:10-07:00", - "end": "2019-03-11T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7af69df3-a78a-41d6-9fd8-2fc36529bc08", - "resource": { - "resourceType": "Observation", - "id": "7af69df3-a78a-41d6-9fd8-2fc36529bc08", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 68.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0fa679e1-97d1-46a4-8afa-cbf6a8f52886", - "resource": { - "resourceType": "Observation", - "id": "0fa679e1-97d1-46a4-8afa-cbf6a8f52886", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:18588a86-37e4-4701-b9a2-722f41b13c6b", - "resource": { - "resourceType": "Observation", - "id": "18588a86-37e4-4701-b9a2-722f41b13c6b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 8.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1fbf8520-879e-4ffc-b33e-bb88e5c1a833", - "resource": { - "resourceType": "Observation", - "id": "1fbf8520-879e-4ffc-b33e-bb88e5c1a833", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 58.512, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7949dbd0-ddf4-4105-aa2e-6d0e7e598a53", - "resource": { - "resourceType": "Observation", - "id": "7949dbd0-ddf4-4105-aa2e-6d0e7e598a53", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 43.730, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:61dd1c35-95dd-4b4a-9ff6-1b9e3a4a8271", - "resource": { - "resourceType": "Observation", - "id": "61dd1c35-95dd-4b4a-9ff6-1b9e3a4a8271", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cc0debc9-3366-49a6-9a9d-5b706adc56c3", - "resource": { - "resourceType": "Observation", - "id": "cc0debc9-3366-49a6-9a9d-5b706adc56c3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 65, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:685abc52-deb4-4ae7-a527-5bf07f875b3d", - "resource": { - "resourceType": "Observation", - "id": "685abc52-deb4-4ae7-a527-5bf07f875b3d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:27d7edef-1085-4075-a8a2-6d3b0e2db061", - "resource": { - "resourceType": "Observation", - "id": "27d7edef-1085-4075-a8a2-6d3b0e2db061", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "effectiveDateTime": "2019-03-11T06:13:10-07:00", - "issued": "2019-03-11T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6280d60b-fcf7-4a00-abc3-54ca2dba7375", - "resource": { - "resourceType": "Immunization", - "id": "6280d60b-fcf7-4a00-abc3-54ca2dba7375", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7329d302-1187-4106-a820-36e7e7fa0be8", - "resource": { - "resourceType": "Immunization", - "id": "7329d302-1187-4106-a820-36e7e7fa0be8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:112db251-367d-4e24-b5f0-74d347c20fc7", - "resource": { - "resourceType": "Immunization", - "id": "112db251-367d-4e24-b5f0-74d347c20fc7", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:5f503634-27b4-4d39-af2d-9616305fb112", - "resource": { - "resourceType": "Immunization", - "id": "5f503634-27b4-4d39-af2d-9616305fb112", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:09afdb46-413f-439e-a4d3-cc17e70540de", - "resource": { - "resourceType": "Immunization", - "id": "09afdb46-413f-439e-a4d3-cc17e70540de", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - }, - "date": "2019-03-11T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c7f1547d-b417-4360-9b4a-17e407b1799f", - "resource": { - "resourceType": "Claim", - "id": "c7f1547d-b417-4360-9b4a-17e407b1799f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-03-11T06:13:10-07:00", - "end": "2019-03-11T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:6280d60b-fcf7-4a00-abc3-54ca2dba7375" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7329d302-1187-4106-a820-36e7e7fa0be8" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:112db251-367d-4e24-b5f0-74d347c20fc7" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:5f503634-27b4-4d39-af2d-9616305fb112" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:09afdb46-413f-439e-a4d3-cc17e70540de" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:cd9fa994-4899-4e33-bcbd-7e526b471578" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a66e53c0-1be1-408d-adf9-7dbf08ce8850", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a66e53c0-1be1-408d-adf9-7dbf08ce8850", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c7f1547d-b417-4360-9b4a-17e407b1799f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-03-11T06:28:10-07:00", - "end": "2020-03-11T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3", - "resource": { - "resourceType": "Encounter", - "id": "dc6f80e7-beff-4b8a-8f62-b3e13ee315b3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2019-06-10T06:13:10-07:00", - "end": "2019-06-10T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:36c0832b-c32a-440a-a794-03f58d056ef4", - "resource": { - "resourceType": "Observation", - "id": "36c0832b-c32a-440a-a794-03f58d056ef4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 72, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2086c21a-a5af-4559-9420-92e13e76c294", - "resource": { - "resourceType": "Observation", - "id": "2086c21a-a5af-4559-9420-92e13e76c294", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:368d1e69-17b8-4c7a-b464-fe9e14711856", - "resource": { - "resourceType": "Observation", - "id": "368d1e69-17b8-4c7a-b464-fe9e14711856", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 9.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9b277d04-6184-4d37-9fea-10c2699fb028", - "resource": { - "resourceType": "Observation", - "id": "9b277d04-6184-4d37-9fea-10c2699fb028", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 59.098, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e2dc12e-15b0-4c52-ad97-9116e8f3dd4f", - "resource": { - "resourceType": "Observation", - "id": "1e2dc12e-15b0-4c52-ad97-9116e8f3dd4f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 44.880, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:73edd7e7-c6c3-4dc9-82b8-d162e2099271", - "resource": { - "resourceType": "Observation", - "id": "73edd7e7-c6c3-4dc9-82b8-d162e2099271", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:78603cd5-5300-401e-b2fe-157efd7b41b3", - "resource": { - "resourceType": "Observation", - "id": "78603cd5-5300-401e-b2fe-157efd7b41b3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 69, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:451028a6-f90c-41de-a525-3047815de7fe", - "resource": { - "resourceType": "Observation", - "id": "451028a6-f90c-41de-a525-3047815de7fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf54424e-c9b0-4c0d-ad50-3014dc5d3667", - "resource": { - "resourceType": "Observation", - "id": "cf54424e-c9b0-4c0d-ad50-3014dc5d3667", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "effectiveDateTime": "2019-06-10T06:13:10-07:00", - "issued": "2019-06-10T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fe4b7981-bd87-4d0b-82d3-c168ba8c448a", - "resource": { - "resourceType": "Procedure", - "id": "fe4b7981-bd87-4d0b-82d3-c168ba8c448a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - }, - "performedPeriod": { - "start": "2019-06-10T06:13:10-07:00", - "end": "2019-06-10T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:bf665789-c5c6-452d-9363-03d077ab69b0", - "resource": { - "resourceType": "Claim", - "id": "bf665789-c5c6-452d-9363-03d077ab69b0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-06-10T06:13:10-07:00", - "end": "2019-06-10T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:fe4b7981-bd87-4d0b-82d3-c168ba8c448a" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:dc6f80e7-beff-4b8a-8f62-b3e13ee315b3" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 690.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:37373884-0dde-4dd8-81c9-4967c8315b53", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "37373884-0dde-4dd8-81c9-4967c8315b53", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bf665789-c5c6-452d-9363-03d077ab69b0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-06-10T06:43:10-07:00", - "end": "2020-06-10T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 138.1, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 552.4, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 690.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 690.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 552.4, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", - "resource": { - "resourceType": "Practitioner", - "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10" - } - ], - "active": true, - "name": [ - { - "family": "Jenkins714", - "given": [ - "Lara964" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71", - "resource": { - "resourceType": "Encounter", - "id": "601f35fa-fb17-4771-a033-d1b624db5a71", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5", - "resource": { - "resourceType": "Condition", - "id": "8a2d671f-09cc-4e92-89aa-27e19adb0ea5", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "241929008", - "display": "Acute allergic reaction" - } - ], - "text": "Acute allergic reaction" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - }, - "onsetDateTime": "2019-08-26T06:13:10-07:00", - "abatementDateTime": "2019-08-26T11:13:10-07:00", - "assertedDate": "2019-08-26T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:cd31a507-9600-4978-b8e2-0fecb70d07a5", - "resource": { - "resourceType": "Procedure", - "id": "cd31a507-9600-4978-b8e2-0fecb70d07a5", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "313191000", - "display": "Injection of adrenaline" - } - ], - "text": "Injection of adrenaline" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - }, - "performedPeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5d47da3e-ada9-48eb-8199-7d1540239fdb", - "resource": { - "resourceType": "MedicationRequest", - "id": "5d47da3e-ada9-48eb-8199-7d1540239fdb", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "312617", - "display": "predniSONE 5 MG Oral Tablet" - } - ], - "text": "predniSONE 5 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - }, - "authoredOn": "2019-08-26T11:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:3e823594-104b-48e5-a2b5-92f5da19f176", - "resource": { - "resourceType": "Claim", - "id": "3e823594-104b-48e5-a2b5-92f5da19f176", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:5d47da3e-ada9-48eb-8199-7d1540239fdb" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - } - ] - } - ], - "total": { - "value": 7.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f5fe4b03-66c7-4f84-84e8-4de5823e9d7d", - "resource": { - "resourceType": "Claim", - "id": "f5fe4b03-66c7-4f84-84e8-4de5823e9d7d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:cd31a507-9600-4978-b8e2-0fecb70d07a5" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "313191000" - } - ] - }, - "net": { - "value": 2333.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5f31fcea-92f1-45dc-9d4a-605208e041a7", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5f31fcea-92f1-45dc-9d4a-605208e041a7", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f5fe4b03-66c7-4f84-84e8-4de5823e9d7d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-08-26T11:28:10-07:00", - "end": "2020-08-26T11:28:10-07:00" - }, - "created": "2019-08-26T11:28:10-07:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:f5fe4b03-66c7-4f84-84e8-4de5823e9d7d" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8a2d671f-09cc-4e92-89aa-27e19adb0ea5" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:601f35fa-fb17-4771-a033-d1b624db5a71" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "313191000" - } - ] - }, - "servicedPeriod": { - "start": "2019-08-26T06:13:10-07:00", - "end": "2019-08-26T11:28:10-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "net": { - "value": 2333.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 466.65200000000004, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1866.6080000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2333.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2333.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1866.6080000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "resource": { - "resourceType": "Organization", - "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAMBRIDGE HEALTH ALLIANCE", - "telecom": [ - { - "system": "phone", - "value": "6176652300" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", - "resource": { - "resourceType": "Practitioner", - "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "40" - } - ], - "active": true, - "name": [ - { - "family": "Gibson10", - "given": [ - "Loretta235" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab", - "resource": { - "resourceType": "Encounter", - "id": "cb9f6247-9e30-4bff-90c1-34d7c66fa6ab", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2019-09-02T06:13:10-07:00", - "end": "2019-09-02T06:28:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8cef7d56-00a9-4758-b00b-2e3625e04acc", - "resource": { - "resourceType": "CarePlan", - "id": "8cef7d56-00a9-4758-b00b-2e3625e04acc", - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "384758001", - "display": "Self-care interventions (procedure)" - } - ], - "text": "Self-care interventions (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab" - }, - "period": { - "start": "2019-09-02T06:13:10-07:00" - }, - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "409002", - "display": "Food allergy diet" - } - ], - "text": "Food allergy diet" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58332002", - "display": "Allergy education" - } - ], - "text": "Allergy education" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "58332002", - "display": "Allergy education" - } - ], - "text": "Allergy education" - }, - "status": "in-progress" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:f554de70-4dcd-4bd2-a95d-1490f2079800", - "resource": { - "resourceType": "Claim", - "id": "f554de70-4dcd-4bd2-a95d-1490f2079800", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-09-02T06:13:10-07:00", - "end": "2019-09-02T06:28:10-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:cb9f6247-9e30-4bff-90c1-34d7c66fa6ab" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9e48ce54-a7e9-448d-9d76-4e522fabfa85", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "9e48ce54-a7e9-448d-9d76-4e522fabfa85", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f554de70-4dcd-4bd2-a95d-1490f2079800" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-09-02T06:28:10-07:00", - "end": "2020-09-02T06:28:10-07:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71", - "resource": { - "resourceType": "Encounter", - "id": "ed001667-14dd-48f1-a5ce-0bfb8b373d71", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2019-09-09T06:13:10-07:00", - "end": "2019-09-09T06:43:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2726d223-c231-4e88-a777-0d040011b25e", - "resource": { - "resourceType": "Observation", - "id": "2726d223-c231-4e88-a777-0d040011b25e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 75.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25144b62-082e-4e72-ad67-fde8c6689436", - "resource": { - "resourceType": "Observation", - "id": "25144b62-082e-4e72-ad67-fde8c6689436", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed796f59-3291-4107-9dca-dc11e97044da", - "resource": { - "resourceType": "Observation", - "id": "ed796f59-3291-4107-9dca-dc11e97044da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 9.9000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32934caf-6089-444c-b980-7a0b89afd141", - "resource": { - "resourceType": "Observation", - "id": "32934caf-6089-444c-b980-7a0b89afd141", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 58.121, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5dd8046f-66cc-4111-a89c-72f4665cf3f6", - "resource": { - "resourceType": "Observation", - "id": "5dd8046f-66cc-4111-a89c-72f4665cf3f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 45.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7753fc67-7ae2-4681-b6bc-abc53286acc4", - "resource": { - "resourceType": "Observation", - "id": "7753fc67-7ae2-4681-b6bc-abc53286acc4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d81f759f-a40b-46bc-9e7b-9b77c368cbdc", - "resource": { - "resourceType": "Observation", - "id": "d81f759f-a40b-46bc-9e7b-9b77c368cbdc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 86, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f950fb54-7057-4898-aa24-03e549d51b0c", - "resource": { - "resourceType": "Observation", - "id": "f950fb54-7057-4898-aa24-03e549d51b0c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82e1d551-0027-4974-ae34-ef54ec72f4c7", - "resource": { - "resourceType": "Observation", - "id": "82e1d551-0027-4974-ae34-ef54ec72f4c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "effectiveDateTime": "2019-09-09T06:13:10-07:00", - "issued": "2019-09-09T06:13:10.107-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:739c6bf8-1a80-40e1-a07e-74da5797c929", - "resource": { - "resourceType": "Procedure", - "id": "739c6bf8-1a80-40e1-a07e-74da5797c929", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "performedPeriod": { - "start": "2019-09-09T06:13:10-07:00", - "end": "2019-09-09T06:28:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:bf9f65db-9e55-490b-a69a-4ca965e7e393", - "resource": { - "resourceType": "Immunization", - "id": "bf9f65db-9e55-490b-a69a-4ca965e7e393", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:169a63ab-2a78-4c8f-a4ca-a665103f02b1", - "resource": { - "resourceType": "Immunization", - "id": "169a63ab-2a78-4c8f-a4ca-a665103f02b1", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:36c925c2-10aa-47fe-8c70-78428322704a", - "resource": { - "resourceType": "Immunization", - "id": "36c925c2-10aa-47fe-8c70-78428322704a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:46dd2af4-dcb4-46a4-9f50-3dfb0e632b88", - "resource": { - "resourceType": "Immunization", - "id": "46dd2af4-dcb4-46a4-9f50-3dfb0e632b88", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:72bf6a64-946e-45bb-9919-94542296ba16", - "resource": { - "resourceType": "Immunization", - "id": "72bf6a64-946e-45bb-9919-94542296ba16", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - }, - "date": "2019-09-09T06:13:10-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4fddfb60-257f-4f89-ac0a-dc1b6ca24076", - "resource": { - "resourceType": "Claim", - "id": "4fddfb60-257f-4f89-ac0a-dc1b6ca24076", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-09-09T06:13:10-07:00", - "end": "2019-09-09T06:43:10-07:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:bf9f65db-9e55-490b-a69a-4ca965e7e393" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:169a63ab-2a78-4c8f-a4ca-a665103f02b1" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:36c925c2-10aa-47fe-8c70-78428322704a" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:46dd2af4-dcb4-46a4-9f50-3dfb0e632b88" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:72bf6a64-946e-45bb-9919-94542296ba16" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:739c6bf8-1a80-40e1-a07e-74da5797c929" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ed001667-14dd-48f1-a5ce-0bfb8b373d71" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 673.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:cbfd86b3-3df7-42d3-9ddb-32e6df99de6d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "cbfd86b3-3df7-42d3-9ddb-32e6df99de6d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4fddfb60-257f-4f89-ac0a-dc1b6ca24076" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-09-09T06:43:10-07:00", - "end": "2020-09-09T06:43:10-07:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 134.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 539.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 673.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 673.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1101.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7", - "resource": { - "resourceType": "Encounter", - "id": "3b95a3d5-eb06-4983-990d-2bcc641a43a7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185347001", - "display": "Encounter for problem" - } - ], - "text": "Encounter for problem" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2019-09-14T06:13:10-07:00", - "end": "2019-09-14T06:49:10-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d4d80701-e1e6-4656-bda3-a1770afb5122", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "d4d80701-e1e6-4656-bda3-a1770afb5122", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "419474003", - "display": "Allergy to mould" - } - ], - "text": "Allergy to mould" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:f8b3b9f1-b2df-4434-9369-227d1a92e18c", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "f8b3b9f1-b2df-4434-9369-227d1a92e18c", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "232350006", - "display": "House dust mite allergy" - } - ], - "text": "House dust mite allergy" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:12b4987e-75d2-457f-88d1-6fb5e04097a8", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "12b4987e-75d2-457f-88d1-6fb5e04097a8", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "232347008", - "display": "Dander (animal) allergy" - } - ], - "text": "Dander (animal) allergy" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:f9c3dd3f-0b25-4c6b-88c1-f5e73213c6dc", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "f9c3dd3f-0b25-4c6b-88c1-f5e73213c6dc", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "418689008", - "display": "Allergy to grass pollen" - } - ], - "text": "Allergy to grass pollen" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:e8b91308-2205-42cd-8b50-0013a67f6c46", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "e8b91308-2205-42cd-8b50-0013a67f6c46", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "425525006", - "display": "Allergy to dairy product" - } - ], - "text": "Allergy to dairy product" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:bf642f63-7298-4026-97d7-e82b1f9e49af", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "bf642f63-7298-4026-97d7-e82b1f9e49af", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "417532002", - "display": "Allergy to fish" - } - ], - "text": "Allergy to fish" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:dbb6d6e6-43cf-4670-967d-35fe90d84775", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "dbb6d6e6-43cf-4670-967d-35fe90d84775", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "91934008", - "display": "Allergy to nut" - } - ], - "text": "Allergy to nut" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:8ef4a099-3c31-42e6-b45d-0e96b2a0545c", - "resource": { - "resourceType": "AllergyIntolerance", - "id": "8ef4a099-3c31-42e6-b45d-0e96b2a0545c", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "type": "allergy", - "category": [ - "food" - ], - "criticality": "low", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "91935009", - "display": "Allergy to peanuts" - } - ], - "text": "Allergy to peanuts" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "assertedDate": "2019-09-14T06:13:10-07:00" - }, - "request": { - "method": "POST", - "url": "AllergyIntolerance" - } - }, - { - "fullUrl": "urn:uuid:8fc57162-5928-44bb-bf29-072747744f61", - "resource": { - "resourceType": "Observation", - "id": "8fc57162-5928-44bb-bf29-072747744f61", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6206-7", - "display": "Peanut IgE Ab in Serum" - } - ], - "text": "Peanut IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 13.815, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6958c9b3-3224-49b5-9cbf-1b1c1562e347", - "resource": { - "resourceType": "Observation", - "id": "6958c9b3-3224-49b5-9cbf-1b1c1562e347", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6273-7", - "display": "Walnut IgE Ab in Serum" - } - ], - "text": "Walnut IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 60.513, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:00383c72-18fe-477b-af20-e8b8a2417ced", - "resource": { - "resourceType": "Observation", - "id": "00383c72-18fe-477b-af20-e8b8a2417ced", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6082-2", - "display": "Codfish IgE Ab in Serum" - } - ], - "text": "Codfish IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 54.222, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:722e972a-9927-4dfd-8fec-d7cc7b19f8da", - "resource": { - "resourceType": "Observation", - "id": "722e972a-9927-4dfd-8fec-d7cc7b19f8da", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6246-3", - "display": "Shrimp IgE Ab in Serum" - } - ], - "text": "Shrimp IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.23565, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1c3ee88-9020-4d6e-8e9d-5b409f0c75e9", - "resource": { - "resourceType": "Observation", - "id": "a1c3ee88-9020-4d6e-8e9d-5b409f0c75e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6276-0", - "display": "Wheat IgE Ab in Serum" - } - ], - "text": "Wheat IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.27006, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b17db831-9797-4135-8cee-1503c1bf0a49", - "resource": { - "resourceType": "Observation", - "id": "b17db831-9797-4135-8cee-1503c1bf0a49", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6106-9", - "display": "Egg white IgE Ab in Serum" - } - ], - "text": "Egg white IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.31736, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbab2cc0-16ce-40bb-9c7b-de6a68b9358b", - "resource": { - "resourceType": "Observation", - "id": "bbab2cc0-16ce-40bb-9c7b-de6a68b9358b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6248-9", - "display": "Soybean IgE Ab in Serum" - } - ], - "text": "Soybean IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.075051, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a8a64a5-5879-4754-99da-488f05e7bd47", - "resource": { - "resourceType": "Observation", - "id": "8a8a64a5-5879-4754-99da-488f05e7bd47", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "7258-7", - "display": "Cow milk IgE Ab in Serum" - } - ], - "text": "Cow milk IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 73.784, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7efeb353-ee73-4c2a-a5bf-ed5350793915", - "resource": { - "resourceType": "Observation", - "id": "7efeb353-ee73-4c2a-a5bf-ed5350793915", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6189-5", - "display": "White oak IgE Ab in Serum" - } - ], - "text": "White oak IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.25997, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e151b09-505a-44d8-a744-536e6446f6c1", - "resource": { - "resourceType": "Observation", - "id": "8e151b09-505a-44d8-a744-536e6446f6c1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6085-5", - "display": "Common Ragweed IgE Ab in Serum" - } - ], - "text": "Common Ragweed IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 13.391, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53005f80-c9c1-4a21-b7c3-1137cfdc69a4", - "resource": { - "resourceType": "Observation", - "id": "53005f80-c9c1-4a21-b7c3-1137cfdc69a4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6833-8", - "display": "Cat dander IgE Ab in Serum" - } - ], - "text": "Cat dander IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 49.410, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c91cad0-029d-407f-98aa-962fcca7b815", - "resource": { - "resourceType": "Observation", - "id": "2c91cad0-029d-407f-98aa-962fcca7b815", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6095-4", - "display": "American house dust mite IgE Ab in Serum" - } - ], - "text": "American house dust mite IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 56.759, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f7c0b965-36ee-480b-ad3d-4c1bc2951532", - "resource": { - "resourceType": "Observation", - "id": "f7c0b965-36ee-480b-ad3d-4c1bc2951532", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6075-6", - "display": "Cladosporium herbarum IgE Ab in Serum" - } - ], - "text": "Cladosporium herbarum IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 17.679, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b009f01-57a4-4836-849b-20cc6dce5386", - "resource": { - "resourceType": "Observation", - "id": "0b009f01-57a4-4836-849b-20cc6dce5386", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6844-5", - "display": "Honey bee IgE Ab in Serum" - } - ], - "text": "Honey bee IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.095614, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0a2e899-b32b-427e-8ccc-c74f584cf6f3", - "resource": { - "resourceType": "Observation", - "id": "b0a2e899-b32b-427e-8ccc-c74f584cf6f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6158-0", - "display": "Latex IgE Ab in Serum" - } - ], - "text": "Latex IgE Ab in Serum" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "effectiveDateTime": "2019-09-14T06:13:10-07:00", - "issued": "2019-09-14T06:13:10.107-07:00", - "valueQuantity": { - "value": 0.18124, - "unit": "kU/L", - "system": "http://unitsofmeasure.org", - "code": "kU/L" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:24928c23-eeda-4edc-8cef-b1e11de40565", - "resource": { - "resourceType": "Procedure", - "id": "24928c23-eeda-4edc-8cef-b1e11de40565", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "395142003", - "display": "Allergy screening test" - } - ], - "text": "Allergy screening test" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "performedPeriod": { - "start": "2019-09-14T06:13:10-07:00", - "end": "2019-09-14T06:34:10-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ab80d554-f980-4306-9c08-5b77069fa93b", - "resource": { - "resourceType": "MedicationRequest", - "id": "ab80d554-f980-4306-9c08-5b77069fa93b", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "997488", - "display": "Fexofenadine hydrochloride 30 MG Oral Tablet" - } - ], - "text": "Fexofenadine hydrochloride 30 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "authoredOn": "2019-09-14T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - }, - "onBehalfOf": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:cba2410d-16d2-43e2-a222-26f385e6724e", - "resource": { - "resourceType": "Claim", - "id": "cba2410d-16d2-43e2-a222-26f385e6724e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-09-14T06:13:10-07:00", - "end": "2019-09-14T06:49:10-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "prescription": { - "reference": "urn:uuid:ab80d554-f980-4306-9c08-5b77069fa93b" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - } - ] - } - ], - "total": { - "value": 20.24, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0327ac93-7e68-46c2-9283-6426749a334c", - "resource": { - "resourceType": "MedicationRequest", - "id": "0327ac93-7e68-46c2-9283-6426749a334c", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1870230", - "display": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" - } - ], - "text": "NDA020800 0.3 ML Epinephrine 1 MG/ML Auto-Injector" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - }, - "authoredOn": "2019-09-14T06:13:10-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - }, - "onBehalfOf": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8fc44a50-9eb4-4614-9ec5-9f75cd32bbd5", - "resource": { - "resourceType": "Claim", - "id": "8fc44a50-9eb4-4614-9ec5-9f75cd32bbd5", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-09-14T06:13:10-07:00", - "end": "2019-09-14T06:49:10-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "prescription": { - "reference": "urn:uuid:0327ac93-7e68-46c2-9283-6426749a334c" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - } - ] - } - ], - "total": { - "value": 237.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:60a4837f-cb91-4a30-a85f-2d1a0b9252a7", - "resource": { - "resourceType": "Claim", - "id": "60a4837f-cb91-4a30-a85f-2d1a0b9252a7", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-09-14T06:13:10-07:00", - "end": "2019-09-14T06:49:10-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:24928c23-eeda-4edc-8cef-b1e11de40565" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3b95a3d5-eb06-4983-990d-2bcc641a43a7" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "395142003" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:96694062-d875-40d7-b3a1-a0aa4f032f05", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "96694062-d875-40d7-b3a1-a0aa4f032f05", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "60a4837f-cb91-4a30-a85f-2d1a0b9252a7" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-09-14T06:49:10-07:00", - "end": "2020-09-14T06:49:10-07:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "395142003" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544", - "resource": { - "resourceType": "Encounter", - "id": "3353a56d-d257-49bc-9040-275c92819544", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499" - } - } - ], - "period": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:43:10-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1e7d21ec-e1c9-45c9-97cc-40da922a294b", - "resource": { - "resourceType": "Observation", - "id": "1e7d21ec-e1c9-45c9-97cc-40da922a294b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 78.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3cce288-7f4f-4737-a519-5faf3ac2230c", - "resource": { - "resourceType": "Observation", - "id": "b3cce288-7f4f-4737-a519-5faf3ac2230c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49c20729-78ac-4475-8e09-b13b3eddcc6f", - "resource": { - "resourceType": "Observation", - "id": "49c20729-78ac-4475-8e09-b13b3eddcc6f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 10.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd79636f-977d-4736-afed-8da9b5bd4302", - "resource": { - "resourceType": "Observation", - "id": "cd79636f-977d-4736-afed-8da9b5bd4302", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 53.218, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4893f58d-6824-4e25-842f-37456823f56a", - "resource": { - "resourceType": "Observation", - "id": "4893f58d-6824-4e25-842f-37456823f56a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 46.310, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:147d2240-53f2-4cd9-86d8-bf240ce5c774", - "resource": { - "resourceType": "Observation", - "id": "147d2240-53f2-4cd9-86d8-bf240ce5c774", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 84, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 131, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1a40cb7-a9ec-4426-9537-73786a3a0653", - "resource": { - "resourceType": "Observation", - "id": "f1a40cb7-a9ec-4426-9537-73786a3a0653", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 97, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fede4f58-46f7-4431-9c41-7ec51ac7e374", - "resource": { - "resourceType": "Observation", - "id": "fede4f58-46f7-4431-9c41-7ec51ac7e374", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f6361e81-ce8f-41b7-b912-c94e72128480", - "resource": { - "resourceType": "Observation", - "id": "f6361e81-ce8f-41b7-b912-c94e72128480", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "effectiveDateTime": "2019-12-09T05:13:10-08:00", - "issued": "2019-12-09T05:13:10.107-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87", - "resource": { - "resourceType": "Procedure", - "id": "f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "performedPeriod": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:28:10-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ba32549f-f9e9-4274-bc4f-9a914f0a8991", - "resource": { - "resourceType": "Immunization", - "id": "ba32549f-f9e9-4274-bc4f-9a914f0a8991", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "encounter": { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - }, - "date": "2019-12-09T05:13:10-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:5bb3d5cb-571d-474d-acd7-f18500b08680", - "resource": { - "resourceType": "Claim", - "id": "5bb3d5cb-571d-474d-acd7-f18500b08680", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-12-09T05:13:10-08:00", - "end": "2019-12-09T05:43:10-08:00" - }, - "organization": { - "reference": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ba32549f-f9e9-4274-bc4f-9a914f0a8991" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:f2d1e1e2-97b3-4d5a-960a-ab0c883dcc87" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3353a56d-d257-49bc-9040-275c92819544" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 318.06, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f17b038c-4245-4a8a-a6cd-4e9d5df76067", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f17b038c-4245-4a8a-a6cd-4e9d5df76067", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Humana" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5bb3d5cb-571d-474d-acd7-f18500b08680" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-12-09T05:43:10-08:00", - "end": "2020-12-09T05:43:10-08:00" - }, - "provider": { - "identifier": { - "value": "d726e757-2975-3d6a-bdf9-517546432499" - } - }, - "organization": { - "identifier": { - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 63.612, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 254.448, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 318.06, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 318.06, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 366.86400000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494", - "resource": { - "resourceType": "Encounter", - "id": "835db05f-836f-4323-bb98-3f330070e494", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2019-12-24T05:13:10-08:00", - "end": "2019-12-24T05:28:10-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7", - "resource": { - "resourceType": "Condition", - "id": "b2631fd3-7099-4f20-b63b-9f6ecc79dec7", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" - }, - "onsetDateTime": "2019-12-24T05:13:10-08:00", - "assertedDate": "2019-12-24T05:13:10-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:d0e6004d-b188-495e-aa52-f68c7681a924", - "resource": { - "resourceType": "MedicationRequest", - "id": "d0e6004d-b188-495e-aa52-f68c7681a924", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "context": { - "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" - }, - "authoredOn": "2019-12-24T05:13:10-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d7530091-8f70-4522-9a9b-df856caacb40", - "resource": { - "resourceType": "Claim", - "id": "d7530091-8f70-4522-9a9b-df856caacb40", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-12-24T05:13:10-08:00", - "end": "2019-12-24T05:28:10-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:d0e6004d-b188-495e-aa52-f68c7681a924" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" - } - ] - } - ], - "total": { - "value": 4.98, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e322a7e0-7755-4420-8d76-e956d8ef64bb", - "resource": { - "resourceType": "Claim", - "id": "e322a7e0-7755-4420-8d76-e956d8ef64bb", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-12-24T05:13:10-08:00", - "end": "2019-12-24T05:28:10-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c0afc820-908b-4b15-a534-d05e037ca605", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c0afc820-908b-4b15-a534-d05e037ca605", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e322a7e0-7755-4420-8d76-e956d8ef64bb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:88fb71f7-d445-4e5b-8af2-962e8f8e5fb6" - }, - "billablePeriod": { - "start": "2019-12-24T05:28:10-08:00", - "end": "2020-12-24T05:28:10-08:00" - }, - "created": "2019-12-24T05:28:10-08:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:e322a7e0-7755-4420-8d76-e956d8ef64bb" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:b2631fd3-7099-4f20-b63b-9f6ecc79dec7" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-12-24T05:13:10-08:00", - "end": "2019-12-24T05:28:10-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:835db05f-836f-4323-bb98-3f330070e494" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-12-24T05:13:10-08:00", - "end": "2019-12-24T05:28:10-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json deleted file mode 100644 index 1f4a6a88e235..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Cris921_Lang846_df27a976-5c5e-4b84-ad00-fe32972dce9c.json +++ /dev/null @@ -1,22688 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c", - "resource": { - "resourceType": "Patient", - "id": "df27a976-5c5e-4b84-ad00-fe32972dce9c", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -4106806494818543385 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2106-3", - "display": "White" - } - }, - { - "url": "text", - "valueString": "White" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Babette571 Dooley940" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Needham", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.003367159185079892 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 5.99663284081492 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-61-1293" - } - ], - "name": [ - { - "use": "official", - "family": "Lang846", - "given": [ - "Cris921" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-281-6453", - "use": "home" - } - ], - "gender": "female", - "birthDate": "2013-11-29", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.09755452320718 - }, - { - "url": "longitude", - "valueDecimal": -70.83495946935349 - } - ] - } - ], - "line": [ - "1082 Vandervort Avenue Apt 70" - ], - "city": "Hanover", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5", - "resource": { - "resourceType": "Organization", - "id": "97066a41-4fd5-3d4e-b382-a978fb20a5d5", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP33234", - "telecom": [ - { - "system": "phone", - "value": "781-829-9300" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.122956 - }, - { - "url": "longitude", - "valueDecimal": -70.85631 - } - ] - } - ], - "line": [ - "198 COLUMBIA RD" - ], - "city": "HANOVER", - "state": "MA", - "postalCode": "02339-2380", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553", - "resource": { - "resourceType": "Practitioner", - "id": "f9f09686-21e4-3b27-ba68-9b7313d9a553", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "22570" - } - ], - "active": true, - "name": [ - { - "family": "Homenick806", - "given": [ - "Evan94" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "198 COLUMBIA RD" - ], - "city": "HANOVER", - "state": "MA", - "postalCode": "02339-2380", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0", - "resource": { - "resourceType": "Encounter", - "id": "de64ec54-276c-4287-b6c1-3584fee7f8b0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2013-11-29T15:21:32-08:00", - "end": "2013-11-29T15:51:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1c1addf3-8cd7-4541-bf69-fce5e21f5346", - "resource": { - "resourceType": "Observation", - "id": "1c1addf3-8cd7-4541-bf69-fce5e21f5346", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 49.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bb059a3a-5c99-48ba-b620-ba10f8f98816", - "resource": { - "resourceType": "Observation", - "id": "bb059a3a-5c99-48ba-b620-ba10f8f98816", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ef19301-2656-4de6-bc0b-40543e493be5", - "resource": { - "resourceType": "Observation", - "id": "0ef19301-2656-4de6-bc0b-40543e493be5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 3.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9a0ddc69-b028-4339-b4a3-a1d4894e774e", - "resource": { - "resourceType": "Observation", - "id": "9a0ddc69-b028-4339-b4a3-a1d4894e774e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 20.501, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc528b99-0633-478b-98ab-6fa2b9d8cd38", - "resource": { - "resourceType": "Observation", - "id": "fc528b99-0633-478b-98ab-6fa2b9d8cd38", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 33.290, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f87afcc1-9ebb-43ac-bd25-0912da6cb3e5", - "resource": { - "resourceType": "Observation", - "id": "f87afcc1-9ebb-43ac-bd25-0912da6cb3e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 128, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34f7c63f-d72a-495a-b640-b3825a84efb1", - "resource": { - "resourceType": "Observation", - "id": "34f7c63f-d72a-495a-b640-b3825a84efb1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:788cf18f-9794-4caa-ab3a-5b6531b8f1ad", - "resource": { - "resourceType": "Observation", - "id": "788cf18f-9794-4caa-ab3a-5b6531b8f1ad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8d773eb-40fc-406c-aea2-02d67e49d897", - "resource": { - "resourceType": "Observation", - "id": "e8d773eb-40fc-406c-aea2-02d67e49d897", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 5.1116, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:94c4633e-60b3-452e-b6e8-0fee0c7a806c", - "resource": { - "resourceType": "Observation", - "id": "94c4633e-60b3-452e-b6e8-0fee0c7a806c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 4.0192, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b699973-0a40-4c98-b8f3-3a5b8823fdb5", - "resource": { - "resourceType": "Observation", - "id": "4b699973-0a40-4c98-b8f3-3a5b8823fdb5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 16.471, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bccd98eb-ded2-409e-911f-a2640ebf6661", - "resource": { - "resourceType": "Observation", - "id": "bccd98eb-ded2-409e-911f-a2640ebf6661", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 41.561, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:75f18ec8-d50a-4489-a1ba-f668473aadf7", - "resource": { - "resourceType": "Observation", - "id": "75f18ec8-d50a-4489-a1ba-f668473aadf7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 86.346, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6c4f772b-2473-49ea-a755-ed2b2b32f570", - "resource": { - "resourceType": "Observation", - "id": "6c4f772b-2473-49ea-a755-ed2b2b32f570", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 30.141, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6f3b27fb-f097-47b9-a24d-6b768568ca24", - "resource": { - "resourceType": "Observation", - "id": "6f3b27fb-f097-47b9-a24d-6b768568ca24", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 35.300, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", - "resource": { - "resourceType": "Observation", - "id": "38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 43.444, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66bc2209-0855-4cda-9dd4-6787afcb0cd0", - "resource": { - "resourceType": "Observation", - "id": "66bc2209-0855-4cda-9dd4-6787afcb0cd0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 442.51, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", - "resource": { - "resourceType": "Observation", - "id": "f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 190.14, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d74fd07e-7aa0-4fc4-9f16-11e8fb682119", - "resource": { - "resourceType": "Observation", - "id": "d74fd07e-7aa0-4fc4-9f16-11e8fb682119", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueQuantity": { - "value": 11.668, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ace82b1-3297-43ab-bef9-a14d931f2837", - "resource": { - "resourceType": "Observation", - "id": "2ace82b1-3297-43ab-bef9-a14d931f2837", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fec49be6-9b1c-44b2-bdf4-1d1f6c884666", - "resource": { - "resourceType": "Procedure", - "id": "fec49be6-9b1c-44b2-bdf4-1d1f6c884666", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "performedPeriod": { - "start": "2013-11-29T15:21:32-08:00", - "end": "2013-11-29T15:36:32-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7257de06-fe3c-462b-8a68-f0de095971e8", - "resource": { - "resourceType": "Immunization", - "id": "7257de06-fe3c-462b-8a68-f0de095971e8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "date": "2013-11-29T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:37349538-615a-43b2-a2c8-23bb185da500", - "resource": { - "resourceType": "DiagnosticReport", - "id": "37349538-615a-43b2-a2c8-23bb185da500", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - }, - "effectiveDateTime": "2013-11-29T15:21:32-08:00", - "issued": "2013-11-29T15:21:32.447-08:00", - "result": [ - { - "reference": "urn:uuid:e8d773eb-40fc-406c-aea2-02d67e49d897", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:94c4633e-60b3-452e-b6e8-0fee0c7a806c", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:4b699973-0a40-4c98-b8f3-3a5b8823fdb5", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:bccd98eb-ded2-409e-911f-a2640ebf6661", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:75f18ec8-d50a-4489-a1ba-f668473aadf7", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:6c4f772b-2473-49ea-a755-ed2b2b32f570", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:6f3b27fb-f097-47b9-a24d-6b768568ca24", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:38baa1aa-38f2-4cbc-8027-6a1b81cc5e0a", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:66bc2209-0855-4cda-9dd4-6787afcb0cd0", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:f17b62f2-8c1e-42a3-8cc7-2ded2178d1a6", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:d74fd07e-7aa0-4fc4-9f16-11e8fb682119", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:cc6b7dd9-633d-404b-a49d-302d27125fdd", - "resource": { - "resourceType": "Claim", - "id": "cc6b7dd9-633d-404b-a49d-302d27125fdd", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2013-11-29T15:21:32-08:00", - "end": "2013-11-29T15:51:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7257de06-fe3c-462b-8a68-f0de095971e8" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:fec49be6-9b1c-44b2-bdf4-1d1f6c884666" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:de64ec54-276c-4287-b6c1-3584fee7f8b0" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 396.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d5243537-4f20-4fb4-98e9-60f75b3c2522", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d5243537-4f20-4fb4-98e9-60f75b3c2522", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cc6b7dd9-633d-404b-a49d-302d27125fdd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-11-29T15:51:32-08:00", - "end": "2014-11-29T15:51:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 79.278, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 317.112, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 396.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 396.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 429.528, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4", - "resource": { - "resourceType": "Encounter", - "id": "28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2014-01-03T15:21:32-08:00", - "end": "2014-01-03T15:51:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2961e432-2b46-4377-8c9b-7da30371f351", - "resource": { - "resourceType": "Observation", - "id": "2961e432-2b46-4377-8c9b-7da30371f351", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 53.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31f23f64-a40b-4821-924d-ed9a8dfb0e79", - "resource": { - "resourceType": "Observation", - "id": "31f23f64-a40b-4821-924d-ed9a8dfb0e79", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d212de82-b1e4-47d4-add4-b90ef5e03379", - "resource": { - "resourceType": "Observation", - "id": "d212de82-b1e4-47d4-add4-b90ef5e03379", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 3.9000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:231d1996-058c-4fdc-a469-66633a7313bc", - "resource": { - "resourceType": "Observation", - "id": "231d1996-058c-4fdc-a469-66633a7313bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 21.188, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9613a409-4a15-4d53-b41a-10c35ef6bcec", - "resource": { - "resourceType": "Observation", - "id": "9613a409-4a15-4d53-b41a-10c35ef6bcec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 36.710, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3928d564-52b0-4c0f-adce-536c8061b234", - "resource": { - "resourceType": "Observation", - "id": "3928d564-52b0-4c0f-adce-536c8061b234", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 87, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7590e440-193b-42ef-8d4c-71d29923bf22", - "resource": { - "resourceType": "Observation", - "id": "7590e440-193b-42ef-8d4c-71d29923bf22", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 75, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f728cf6-7135-44fe-b9cb-0e63343e32e6", - "resource": { - "resourceType": "Observation", - "id": "1f728cf6-7135-44fe-b9cb-0e63343e32e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:05bda50c-3e77-4aa4-b854-5f411685578f", - "resource": { - "resourceType": "Observation", - "id": "05bda50c-3e77-4aa4-b854-5f411685578f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "effectiveDateTime": "2014-01-03T15:21:32-08:00", - "issued": "2014-01-03T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a58560af-b5ab-48b5-b465-59d9736c8a21", - "resource": { - "resourceType": "Procedure", - "id": "a58560af-b5ab-48b5-b465-59d9736c8a21", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "performedPeriod": { - "start": "2014-01-03T15:21:32-08:00", - "end": "2014-01-03T15:36:32-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:82baa6f9-c1d0-4db2-9d46-07d92575974a", - "resource": { - "resourceType": "Immunization", - "id": "82baa6f9-c1d0-4db2-9d46-07d92575974a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - }, - "date": "2014-01-03T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7b398516-ac55-4ca7-9beb-3cd314e23d28", - "resource": { - "resourceType": "Claim", - "id": "7b398516-ac55-4ca7-9beb-3cd314e23d28", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-01-03T15:21:32-08:00", - "end": "2014-01-03T15:51:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:82baa6f9-c1d0-4db2-9d46-07d92575974a" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a58560af-b5ab-48b5-b465-59d9736c8a21" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:28bcbf9b-bc2a-457e-80d8-e2a188fbcfe4" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 648.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:03311493-acff-48c7-9cdc-736fc416bfb6", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "03311493-acff-48c7-9cdc-736fc416bfb6", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7b398516-ac55-4ca7-9beb-3cd314e23d28" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-01-03T15:51:32-08:00", - "end": "2015-01-03T15:51:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 129.70000000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 518.8000000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 648.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 648.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 631.2160000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567", - "resource": { - "resourceType": "Encounter", - "id": "abda8627-8b93-49d9-ac98-e7623872c567", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2014-03-07T15:21:32-08:00", - "end": "2014-03-07T15:36:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7416c609-f9e2-4c37-80b5-e87bd022abec", - "resource": { - "resourceType": "Observation", - "id": "7416c609-f9e2-4c37-80b5-e87bd022abec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 58.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a796ce16-4a5e-4d3e-bfd8-af3431b8b760", - "resource": { - "resourceType": "Observation", - "id": "a796ce16-4a5e-4d3e-bfd8-af3431b8b760", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b7b94a8-db76-492a-8b40-2617a2d0d69c", - "resource": { - "resourceType": "Observation", - "id": "4b7b94a8-db76-492a-8b40-2617a2d0d69c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 5.1000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:70e80a37-4ca2-40ce-a530-3cac6c0ef202", - "resource": { - "resourceType": "Observation", - "id": "70e80a37-4ca2-40ce-a530-3cac6c0ef202", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 22.579, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fecdbf07-2ca3-48be-95e4-a1e25abe3bce", - "resource": { - "resourceType": "Observation", - "id": "fecdbf07-2ca3-48be-95e4-a1e25abe3bce", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 39.290, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db07f377-0b5d-4174-803e-f1801c801e7a", - "resource": { - "resourceType": "Observation", - "id": "db07f377-0b5d-4174-803e-f1801c801e7a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e9bfc74c-7285-4bbc-a92f-ca4739fb7779", - "resource": { - "resourceType": "Observation", - "id": "e9bfc74c-7285-4bbc-a92f-ca4739fb7779", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 64, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35580509-4731-42b6-8c25-61ec3485bf83", - "resource": { - "resourceType": "Observation", - "id": "35580509-4731-42b6-8c25-61ec3485bf83", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac3a0166-cd40-41b3-aae4-1b57e90eb5ed", - "resource": { - "resourceType": "Observation", - "id": "ac3a0166-cd40-41b3-aae4-1b57e90eb5ed", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "effectiveDateTime": "2014-03-07T15:21:32-08:00", - "issued": "2014-03-07T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f6b147df-1285-44f2-9a30-b9ec95f8f820", - "resource": { - "resourceType": "Immunization", - "id": "f6b147df-1285-44f2-9a30-b9ec95f8f820", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "date": "2014-03-07T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:04354efd-8ddb-4d5d-a51e-f729a1c64c6b", - "resource": { - "resourceType": "Immunization", - "id": "04354efd-8ddb-4d5d-a51e-f729a1c64c6b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "date": "2014-03-07T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:91191980-ecd5-4e81-b1bd-819fdff84f37", - "resource": { - "resourceType": "Immunization", - "id": "91191980-ecd5-4e81-b1bd-819fdff84f37", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "date": "2014-03-07T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:966ccb19-d147-47f9-a369-c77bf8cc96cc", - "resource": { - "resourceType": "Immunization", - "id": "966ccb19-d147-47f9-a369-c77bf8cc96cc", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "date": "2014-03-07T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:dc0b0d2a-9fda-4df8-baf7-20c14b644237", - "resource": { - "resourceType": "Immunization", - "id": "dc0b0d2a-9fda-4df8-baf7-20c14b644237", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - }, - "date": "2014-03-07T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:91ec0186-757e-4933-a46e-be9fb938bd43", - "resource": { - "resourceType": "Claim", - "id": "91ec0186-757e-4933-a46e-be9fb938bd43", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-03-07T15:21:32-08:00", - "end": "2014-03-07T15:36:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f6b147df-1285-44f2-9a30-b9ec95f8f820" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:04354efd-8ddb-4d5d-a51e-f729a1c64c6b" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:91191980-ecd5-4e81-b1bd-819fdff84f37" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:966ccb19-d147-47f9-a369-c77bf8cc96cc" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:dc0b0d2a-9fda-4df8-baf7-20c14b644237" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:abda8627-8b93-49d9-ac98-e7623872c567" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dce04e16-40c4-4f17-87f7-40a3b12ec920", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "dce04e16-40c4-4f17-87f7-40a3b12ec920", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "91ec0186-757e-4933-a46e-be9fb938bd43" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-03-07T15:36:32-08:00", - "end": "2015-03-07T15:36:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad", - "resource": { - "resourceType": "Encounter", - "id": "ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2014-05-09T16:21:32-07:00", - "end": "2014-05-09T16:51:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:dd43d232-e6f0-4c33-8844-db440c3ca4c8", - "resource": { - "resourceType": "Observation", - "id": "dd43d232-e6f0-4c33-8844-db440c3ca4c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 62.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f128b0e5-15c5-4828-8754-ca1999bd746f", - "resource": { - "resourceType": "Observation", - "id": "f128b0e5-15c5-4828-8754-ca1999bd746f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7f66a421-5ccf-4573-82f4-0878643e39fe", - "resource": { - "resourceType": "Observation", - "id": "7f66a421-5ccf-4573-82f4-0878643e39fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 6.1000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e53a668-41f2-4af9-9d99-46c565fd2e2e", - "resource": { - "resourceType": "Observation", - "id": "4e53a668-41f2-4af9-9d99-46c565fd2e2e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 25.754, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c11f2f6-7f75-4e2e-9399-1539de7b5c29", - "resource": { - "resourceType": "Observation", - "id": "2c11f2f6-7f75-4e2e-9399-1539de7b5c29", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 40.940, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:95cbf501-33fb-4a35-b86a-8e35f814b17d", - "resource": { - "resourceType": "Observation", - "id": "95cbf501-33fb-4a35-b86a-8e35f814b17d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb516dbb-be5f-4c25-8e3b-a717c7cd45dd", - "resource": { - "resourceType": "Observation", - "id": "cb516dbb-be5f-4c25-8e3b-a717c7cd45dd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:305c80be-919b-4b6a-8727-39d92620ad0f", - "resource": { - "resourceType": "Observation", - "id": "305c80be-919b-4b6a-8727-39d92620ad0f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49a7bae4-0919-4112-9c0f-83925b2d902c", - "resource": { - "resourceType": "Observation", - "id": "49a7bae4-0919-4112-9c0f-83925b2d902c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "effectiveDateTime": "2014-05-09T16:21:32-07:00", - "issued": "2014-05-09T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e4a2c1d-813e-4206-b353-00ece4b65402", - "resource": { - "resourceType": "Procedure", - "id": "5e4a2c1d-813e-4206-b353-00ece4b65402", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "performedPeriod": { - "start": "2014-05-09T16:21:32-07:00", - "end": "2014-05-09T16:36:32-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34", - "resource": { - "resourceType": "Immunization", - "id": "7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "date": "2014-05-09T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1", - "resource": { - "resourceType": "Immunization", - "id": "2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "date": "2014-05-09T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b80bd7af-769c-4b52-8290-3a6ed69f545c", - "resource": { - "resourceType": "Immunization", - "id": "b80bd7af-769c-4b52-8290-3a6ed69f545c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "date": "2014-05-09T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c9a3c887-4bff-4442-b081-db928259ef2a", - "resource": { - "resourceType": "Immunization", - "id": "c9a3c887-4bff-4442-b081-db928259ef2a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "date": "2014-05-09T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:45f69ce9-e06d-425a-a90b-927cb5d34e6d", - "resource": { - "resourceType": "Immunization", - "id": "45f69ce9-e06d-425a-a90b-927cb5d34e6d", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - }, - "date": "2014-05-09T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e5381efa-2d2e-441b-985c-faebe470b15a", - "resource": { - "resourceType": "Claim", - "id": "e5381efa-2d2e-441b-985c-faebe470b15a", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-05-09T16:21:32-07:00", - "end": "2014-05-09T16:51:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7cdb64c6-b1b6-4b13-b2ed-a3f04f5ada34" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:2d70c2ee-5ec8-4eba-b9dc-c059b0cf0bf1" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:b80bd7af-769c-4b52-8290-3a6ed69f545c" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c9a3c887-4bff-4442-b081-db928259ef2a" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:45f69ce9-e06d-425a-a90b-927cb5d34e6d" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:5e4a2c1d-813e-4206-b353-00ece4b65402" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ded69b0f-f3d2-401b-b4b1-9bb963a4e0ad" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 831.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3a5031aa-765e-4ec0-a6e6-12ef90c18dc8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3a5031aa-765e-4ec0-a6e6-12ef90c18dc8", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e5381efa-2d2e-441b-985c-faebe470b15a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-05-09T16:51:32-07:00", - "end": "2015-05-09T16:51:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 166.286, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 665.144, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 831.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 831.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1227.2240000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b", - "resource": { - "resourceType": "Encounter", - "id": "43174bce-2315-4d09-a84c-953ed0d9385b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2014-08-08T16:21:32-07:00", - "end": "2014-08-08T16:51:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:362a225c-da34-43d9-926f-e4105eb3dfb5", - "resource": { - "resourceType": "Observation", - "id": "362a225c-da34-43d9-926f-e4105eb3dfb5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 66.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25c13fd9-fc4f-4e0a-9228-45bb8e4a12e4", - "resource": { - "resourceType": "Observation", - "id": "25c13fd9-fc4f-4e0a-9228-45bb8e4a12e4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:16c09d37-af4b-43cc-af83-03edcace4551", - "resource": { - "resourceType": "Observation", - "id": "16c09d37-af4b-43cc-af83-03edcace4551", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 7.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fd997161-2225-4582-8b69-8a349b78026a", - "resource": { - "resourceType": "Observation", - "id": "fd997161-2225-4582-8b69-8a349b78026a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 32.896, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3694ae02-1942-4900-9042-99510a918435", - "resource": { - "resourceType": "Observation", - "id": "3694ae02-1942-4900-9042-99510a918435", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 42.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba618b9a-a15a-4067-ae4d-1301aacf133a", - "resource": { - "resourceType": "Observation", - "id": "ba618b9a-a15a-4067-ae4d-1301aacf133a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33229ce8-48fd-447f-ab2f-a6ba8da654fd", - "resource": { - "resourceType": "Observation", - "id": "33229ce8-48fd-447f-ab2f-a6ba8da654fd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 96, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b7d305e-f91e-4200-97c9-1b8c6e203f0d", - "resource": { - "resourceType": "Observation", - "id": "4b7d305e-f91e-4200-97c9-1b8c6e203f0d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f74d9df1-af18-43c1-b6a5-e15a827ebbdc", - "resource": { - "resourceType": "Observation", - "id": "f74d9df1-af18-43c1-b6a5-e15a827ebbdc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "effectiveDateTime": "2014-08-08T16:21:32-07:00", - "issued": "2014-08-08T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9", - "resource": { - "resourceType": "Procedure", - "id": "2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "performedPeriod": { - "start": "2014-08-08T16:21:32-07:00", - "end": "2014-08-08T16:36:32-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ed53940a-e1f5-46f3-ac61-de1af1fbab7c", - "resource": { - "resourceType": "Immunization", - "id": "ed53940a-e1f5-46f3-ac61-de1af1fbab7c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "date": "2014-08-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:74132144-c817-4926-834b-20e10eacd426", - "resource": { - "resourceType": "Immunization", - "id": "74132144-c817-4926-834b-20e10eacd426", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "date": "2014-08-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:659b9e5b-365e-415f-bc81-be05f53f23ec", - "resource": { - "resourceType": "Immunization", - "id": "659b9e5b-365e-415f-bc81-be05f53f23ec", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "date": "2014-08-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:180a9ca7-a4b3-4215-b689-395169136224", - "resource": { - "resourceType": "Immunization", - "id": "180a9ca7-a4b3-4215-b689-395169136224", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "date": "2014-08-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:141770a5-bd98-4aa9-8e74-6971b14ade5a", - "resource": { - "resourceType": "Immunization", - "id": "141770a5-bd98-4aa9-8e74-6971b14ade5a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - }, - "date": "2014-08-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:dae75b9c-3d5b-463f-baa0-2e33998b103d", - "resource": { - "resourceType": "Claim", - "id": "dae75b9c-3d5b-463f-baa0-2e33998b103d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-08-08T16:21:32-07:00", - "end": "2014-08-08T16:51:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ed53940a-e1f5-46f3-ac61-de1af1fbab7c" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:74132144-c817-4926-834b-20e10eacd426" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:659b9e5b-365e-415f-bc81-be05f53f23ec" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:180a9ca7-a4b3-4215-b689-395169136224" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:141770a5-bd98-4aa9-8e74-6971b14ade5a" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:2734ffdf-c6b1-455c-8aa0-eb3f0a2025a9" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:43174bce-2315-4d09-a84c-953ed0d9385b" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 732.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:00be94df-3f1d-454c-bd48-31e39cdd0544", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "00be94df-3f1d-454c-bd48-31e39cdd0544", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "dae75b9c-3d5b-463f-baa0-2e33998b103d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-08-08T16:51:32-07:00", - "end": "2015-08-08T16:51:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 146.504, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 586.016, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 732.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 732.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1148.096, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5", - "resource": { - "resourceType": "Encounter", - "id": "a62ab6f4-b2c3-4bec-ae76-9213db8c54d5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2014-11-07T15:21:32-08:00", - "end": "2014-11-07T15:51:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:be670ff4-78cb-4000-8935-371664a36a5c", - "resource": { - "resourceType": "Observation", - "id": "be670ff4-78cb-4000-8935-371664a36a5c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 70.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:93402cf4-52b0-476f-8b1d-5ed86d07022d", - "resource": { - "resourceType": "Observation", - "id": "93402cf4-52b0-476f-8b1d-5ed86d07022d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:055519a3-6d32-40de-8935-d3cb3b24d9ae", - "resource": { - "resourceType": "Observation", - "id": "055519a3-6d32-40de-8935-d3cb3b24d9ae", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 8.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d5198a4b-c61c-4571-9146-0b99de769332", - "resource": { - "resourceType": "Observation", - "id": "d5198a4b-c61c-4571-9146-0b99de769332", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 36.840, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7aa4496-f8e4-49aa-9ae6-e490e290954e", - "resource": { - "resourceType": "Observation", - "id": "b7aa4496-f8e4-49aa-9ae6-e490e290954e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 43.730, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d463a6c7-90b5-4204-8b58-48e60342fe24", - "resource": { - "resourceType": "Observation", - "id": "d463a6c7-90b5-4204-8b58-48e60342fe24", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 73, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 131, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c0b7b28e-2a6a-4ba7-a031-8036203855bd", - "resource": { - "resourceType": "Observation", - "id": "c0b7b28e-2a6a-4ba7-a031-8036203855bd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 77, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c2a4b1a0-3709-4020-aeb8-9d8acc017ffb", - "resource": { - "resourceType": "Observation", - "id": "c2a4b1a0-3709-4020-aeb8-9d8acc017ffb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a05af24-5448-4ac4-9c7e-e99eb7f0d202", - "resource": { - "resourceType": "Observation", - "id": "5a05af24-5448-4ac4-9c7e-e99eb7f0d202", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "effectiveDateTime": "2014-11-07T15:21:32-08:00", - "issued": "2014-11-07T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6295f137-97aa-4765-803e-1680110779dc", - "resource": { - "resourceType": "Procedure", - "id": "6295f137-97aa-4765-803e-1680110779dc", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - }, - "performedPeriod": { - "start": "2014-11-07T15:21:32-08:00", - "end": "2014-11-07T15:36:32-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3d05bc39-e280-4dab-8a71-3f5820714393", - "resource": { - "resourceType": "Claim", - "id": "3d05bc39-e280-4dab-8a71-3f5820714393", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-11-07T15:21:32-08:00", - "end": "2014-11-07T15:51:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:6295f137-97aa-4765-803e-1680110779dc" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:a62ab6f4-b2c3-4bec-ae76-9213db8c54d5" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 702.98, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a476e888-3e52-4220-abbb-b29d018c6af6", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a476e888-3e52-4220-abbb-b29d018c6af6", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3d05bc39-e280-4dab-8a71-3f5820714393" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-11-07T15:51:32-08:00", - "end": "2015-11-07T15:51:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 140.596, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 562.384, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 702.98, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 702.98, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.384, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", - "resource": { - "resourceType": "Organization", - "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "GOOD SAMARITAN MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5084273000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "resource": { - "resourceType": "Practitioner", - "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "460" - } - ], - "active": true, - "name": [ - { - "family": "Fay398", - "given": [ - "Deneen201" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613", - "resource": { - "resourceType": "Encounter", - "id": "5daa0556-37d5-4c15-af85-f14841b26613", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2014-11-26T15:21:32-08:00", - "end": "2014-11-26T15:42:32-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796", - "resource": { - "resourceType": "Condition", - "id": "ad242a39-266b-4164-91ca-232fd24a7796", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - }, - "onsetDateTime": "2014-11-26T15:21:32-08:00", - "abatementDateTime": "2014-12-10T15:21:32-08:00", - "assertedDate": "2014-11-26T15:21:32-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:e77d1595-3949-4c37-875f-f1cbec1b8aa5", - "resource": { - "resourceType": "Procedure", - "id": "e77d1595-3949-4c37-875f-f1cbec1b8aa5", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "269911007", - "display": "Sputum examination (procedure)" - } - ], - "text": "Sputum examination (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - }, - "performedPeriod": { - "start": "2014-11-26T15:21:32-08:00", - "end": "2014-11-26T15:27:32-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796", - "display": "Acute bronchitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:164b333c-2481-44f9-8c82-cbd92b873de8", - "resource": { - "resourceType": "MedicationRequest", - "id": "164b333c-2481-44f9-8c82-cbd92b873de8", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - }, - "authoredOn": "2014-11-26T15:21:32-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - }, - "onBehalfOf": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d8df90cb-d960-45d9-b2d0-8dd468394021", - "resource": { - "resourceType": "Claim", - "id": "d8df90cb-d960-45d9-b2d0-8dd468394021", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-11-26T15:21:32-08:00", - "end": "2014-11-26T15:42:32-08:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "prescription": { - "reference": "urn:uuid:164b333c-2481-44f9-8c82-cbd92b873de8" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - } - ] - } - ], - "total": { - "value": 5.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d42bf95c-51de-42e6-9f80-17621516592d", - "resource": { - "resourceType": "CarePlan", - "id": "d42bf95c-51de-42e6-9f80-17621516592d", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - }, - "period": { - "start": "2014-11-26T15:21:32-08:00", - "end": "2015-02-06T15:21:32-08:00" - }, - "addresses": [ - { - "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:cbbe8272-8363-4243-bc2b-511901299496", - "resource": { - "resourceType": "Claim", - "id": "cbbe8272-8363-4243-bc2b-511901299496", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-11-26T15:21:32-08:00", - "end": "2014-11-26T15:42:32-08:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e77d1595-3949-4c37-875f-f1cbec1b8aa5" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5daa0556-37d5-4c15-af85-f14841b26613" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "269911007" - } - ] - }, - "net": { - "value": 5483.95, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:aab3e329-346e-4c87-a029-b3f11d14840a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "aab3e329-346e-4c87-a029-b3f11d14840a", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "cbbe8272-8363-4243-bc2b-511901299496" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-11-26T15:42:32-08:00", - "end": "2015-11-26T15:42:32-08:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:ad242a39-266b-4164-91ca-232fd24a7796" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "269911007" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 1096.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 4387.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 5483.95, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 5483.95, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 4387.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", - "resource": { - "resourceType": "Organization", - "id": "c44f361c-2efb-3050-8f97-0354a12e2920", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "c44f361c-2efb-3050-8f97-0354a12e2920" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5089417000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", - "resource": { - "resourceType": "Practitioner", - "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "200" - } - ], - "active": true, - "name": [ - { - "family": "Padberg411", - "given": [ - "Nila48" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb", - "resource": { - "resourceType": "Encounter", - "id": "cf852eb0-483e-428d-9271-486e9298b0fb", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" - } - } - ], - "period": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e4025922-c623-481b-a3c7-4e7947dce52d", - "resource": { - "resourceType": "Observation", - "id": "e4025922-c623-481b-a3c7-4e7947dce52d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 71.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:98a2275b-6b81-44c0-93bd-4d3bcf5bcb39", - "resource": { - "resourceType": "Observation", - "id": "98a2275b-6b81-44c0-93bd-4d3bcf5bcb39", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e57d90c2-71be-4c09-8a2a-77782b41a557", - "resource": { - "resourceType": "Observation", - "id": "e57d90c2-71be-4c09-8a2a-77782b41a557", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 8.7000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0651705c-6d2a-40b6-877f-e1d7492bade3", - "resource": { - "resourceType": "Observation", - "id": "0651705c-6d2a-40b6-877f-e1d7492bade3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 40.627, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32ce8504-bc8c-4e60-87ef-a91f35e51f1d", - "resource": { - "resourceType": "Observation", - "id": "32ce8504-bc8c-4e60-87ef-a91f35e51f1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 44.040, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36196780-3273-43f8-9404-13184bc36ad5", - "resource": { - "resourceType": "Observation", - "id": "36196780-3273-43f8-9404-13184bc36ad5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 118, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f531201b-6789-4db0-9e70-0371f37c01e7", - "resource": { - "resourceType": "Observation", - "id": "f531201b-6789-4db0-9e70-0371f37c01e7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c036e7c5-bc6b-4afb-abea-077f0287a423", - "resource": { - "resourceType": "Observation", - "id": "c036e7c5-bc6b-4afb-abea-077f0287a423", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d4540e5-371d-4d77-a378-1d8dd891a963", - "resource": { - "resourceType": "Observation", - "id": "6d4540e5-371d-4d77-a378-1d8dd891a963", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "effectiveDateTime": "2014-12-05T15:21:32-08:00", - "issued": "2014-12-05T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9dce2503-5576-49bc-b0df-a1d98ad4bc11", - "resource": { - "resourceType": "Immunization", - "id": "9dce2503-5576-49bc-b0df-a1d98ad4bc11", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "date": "2014-12-05T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007", - "resource": { - "resourceType": "Immunization", - "id": "e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "date": "2014-12-05T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:664cd96e-79b4-459c-8ba2-85d25bf84108", - "resource": { - "resourceType": "Immunization", - "id": "664cd96e-79b4-459c-8ba2-85d25bf84108", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "date": "2014-12-05T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a3d039d6-dd93-4d43-a741-6be2ee312375", - "resource": { - "resourceType": "Immunization", - "id": "a3d039d6-dd93-4d43-a741-6be2ee312375", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "date": "2014-12-05T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:81205a66-b680-4861-b1de-eb0ba1881202", - "resource": { - "resourceType": "Immunization", - "id": "81205a66-b680-4861-b1de-eb0ba1881202", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - }, - "date": "2014-12-05T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:eb3fe20b-3b1d-43c9-af43-882fc0e8b576", - "resource": { - "resourceType": "Claim", - "id": "eb3fe20b-3b1d-43c9-af43-882fc0e8b576", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "organization": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:9dce2503-5576-49bc-b0df-a1d98ad4bc11" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e3ae5300-f9e2-4c1a-a26d-93a6aa3ce007" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:664cd96e-79b4-459c-8ba2-85d25bf84108" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a3d039d6-dd93-4d43-a741-6be2ee312375" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:81205a66-b680-4861-b1de-eb0ba1881202" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f16e3769-5904-4842-b69a-a256ada215d1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f16e3769-5904-4842-b69a-a256ada215d1", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "eb3fe20b-3b1d-43c9-af43-882fc0e8b576" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2014-12-05T15:36:32-08:00", - "end": "2015-12-05T15:36:32-08:00" - }, - "created": "2014-12-05T15:36:32-08:00", - "provider": { - "identifier": { - "value": "d08d5908-40a4-3571-8c54-0cc029c8277f" - } - }, - "organization": { - "identifier": { - "value": "c44f361c-2efb-3050-8f97-0354a12e2920" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:eb3fe20b-3b1d-43c9-af43-882fc0e8b576" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:cf852eb0-483e-428d-9271-486e9298b0fb" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-12-05T15:21:32-08:00", - "end": "2014-12-05T15:36:32-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0", - "resource": { - "resourceType": "Encounter", - "id": "dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2015-02-06T15:21:32-08:00", - "end": "2015-02-06T15:51:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ca2090e9-efe2-4c5d-8919-8bdf6012e351", - "resource": { - "resourceType": "Observation", - "id": "ca2090e9-efe2-4c5d-8919-8bdf6012e351", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 73.900, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:633fae99-bec8-4c26-b5d5-a9648508ef07", - "resource": { - "resourceType": "Observation", - "id": "633fae99-bec8-4c26-b5d5-a9648508ef07", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb1527bb-5418-4ec0-9b53-0d74faafd2e2", - "resource": { - "resourceType": "Observation", - "id": "cb1527bb-5418-4ec0-9b53-0d74faafd2e2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 9.2000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3736248b-6e4f-48b6-bd27-e31e42dc3c7e", - "resource": { - "resourceType": "Observation", - "id": "3736248b-6e4f-48b6-bd27-e31e42dc3c7e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 42.908, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:41c607fd-4915-4f99-91c5-74d9995a2a1a", - "resource": { - "resourceType": "Observation", - "id": "41c607fd-4915-4f99-91c5-74d9995a2a1a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 44.570, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:210d1e9b-451c-499a-9f33-39243968b227", - "resource": { - "resourceType": "Observation", - "id": "210d1e9b-451c-499a-9f33-39243968b227", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 117, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c1587e68-1237-41ec-9d4b-4be23b1ef8a3", - "resource": { - "resourceType": "Observation", - "id": "c1587e68-1237-41ec-9d4b-4be23b1ef8a3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 83, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6f26a56-0eec-4790-a9f1-970a363c9ebd", - "resource": { - "resourceType": "Observation", - "id": "c6f26a56-0eec-4790-a9f1-970a363c9ebd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:48a77809-fe5e-4ad9-aab8-aa5975b473a5", - "resource": { - "resourceType": "Observation", - "id": "48a77809-fe5e-4ad9-aab8-aa5975b473a5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "effectiveDateTime": "2015-02-06T15:21:32-08:00", - "issued": "2015-02-06T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1", - "resource": { - "resourceType": "Procedure", - "id": "4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - }, - "performedPeriod": { - "start": "2015-02-06T15:21:32-08:00", - "end": "2015-02-06T15:36:32-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f7fe2351-68e7-41e2-ba8f-b23db05c525c", - "resource": { - "resourceType": "Claim", - "id": "f7fe2351-68e7-41e2-ba8f-b23db05c525c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2015-02-06T15:21:32-08:00", - "end": "2015-02-06T15:51:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4e73dc82-8e0a-4e56-bf69-ec0d74c8cdc1" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:dd9ae7e6-e578-4c2c-9ea4-fa43da1f11a0" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 608.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:72c5cafb-5fe0-40e0-a8ea-c6cd4b932ce5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "72c5cafb-5fe0-40e0-a8ea-c6cd4b932ce5", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f7fe2351-68e7-41e2-ba8f-b23db05c525c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-02-06T15:51:32-08:00", - "end": "2016-02-06T15:51:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 121.79200000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 487.16800000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 608.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 608.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 487.16800000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777", - "resource": { - "resourceType": "Encounter", - "id": "9df7d2be-be25-4a9e-a85a-c6955ab53777", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2015-05-08T16:21:32-07:00", - "end": "2015-05-08T16:51:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:29c8089f-303b-486b-984a-4c91269d1f3e", - "resource": { - "resourceType": "Observation", - "id": "29c8089f-303b-486b-984a-4c91269d1f3e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 76.900, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c67927ad-7b8e-4e3f-ae5a-2996c9e73f04", - "resource": { - "resourceType": "Observation", - "id": "c67927ad-7b8e-4e3f-ae5a-2996c9e73f04", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b9a51ae-dff4-40a4-9841-f4360f75d47d", - "resource": { - "resourceType": "Observation", - "id": "4b9a51ae-dff4-40a4-9841-f4360f75d47d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 9.8000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bb8ba01-7438-40f1-b4a3-10888c92a658", - "resource": { - "resourceType": "Observation", - "id": "7bb8ba01-7438-40f1-b4a3-10888c92a658", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 39.248, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ce22474b-e062-4cea-923e-81890efbafe9", - "resource": { - "resourceType": "Observation", - "id": "ce22474b-e062-4cea-923e-81890efbafe9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 45.210, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c008671-044e-42ae-8a20-eb10e727942a", - "resource": { - "resourceType": "Observation", - "id": "1c008671-044e-42ae-8a20-eb10e727942a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a1d4bed3-7f4d-4956-9bc7-b97e26f9d8b1", - "resource": { - "resourceType": "Observation", - "id": "a1d4bed3-7f4d-4956-9bc7-b97e26f9d8b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 80, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b254991b-a739-415f-98fa-67f214277b38", - "resource": { - "resourceType": "Observation", - "id": "b254991b-a739-415f-98fa-67f214277b38", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50b79e2d-80c5-4253-bd1e-e01def48e309", - "resource": { - "resourceType": "Observation", - "id": "50b79e2d-80c5-4253-bd1e-e01def48e309", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "effectiveDateTime": "2015-05-08T16:21:32-07:00", - "issued": "2015-05-08T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe", - "resource": { - "resourceType": "Procedure", - "id": "edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "performedPeriod": { - "start": "2015-05-08T16:21:32-07:00", - "end": "2015-05-08T16:36:32-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:8288b96b-ce1c-48c2-b4e0-d1ef2b27851e", - "resource": { - "resourceType": "Immunization", - "id": "8288b96b-ce1c-48c2-b4e0-d1ef2b27851e", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - }, - "date": "2015-05-08T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:caa7a6f1-01f5-417c-8f51-38190b61c434", - "resource": { - "resourceType": "Claim", - "id": "caa7a6f1-01f5-417c-8f51-38190b61c434", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2015-05-08T16:21:32-07:00", - "end": "2015-05-08T16:51:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8288b96b-ce1c-48c2-b4e0-d1ef2b27851e" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:edb36065-4ec1-42c8-b0c3-4c22d3d8b9fe" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:9df7d2be-be25-4a9e-a85a-c6955ab53777" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 795.44, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2764a79a-18f8-49ba-863c-6bd2b7b2bda7", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2764a79a-18f8-49ba-863c-6bd2b7b2bda7", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "caa7a6f1-01f5-417c-8f51-38190b61c434" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-05-08T16:51:32-07:00", - "end": "2016-05-08T16:51:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 159.08800000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 636.3520000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 795.44, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 795.44, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 748.7680000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e", - "resource": { - "resourceType": "Encounter", - "id": "02276df4-3e3e-44ff-87ae-4151d6cfdc6e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2015-11-06T15:21:32-08:00", - "end": "2015-11-06T15:36:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2464911c-0a6b-450e-9e79-157b90b3bd1d", - "resource": { - "resourceType": "Observation", - "id": "2464911c-0a6b-450e-9e79-157b90b3bd1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 82.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed19c96e-d1fd-48ea-8fa7-649e7c29919b", - "resource": { - "resourceType": "Observation", - "id": "ed19c96e-d1fd-48ea-8fa7-649e7c29919b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76e59a72-a598-459f-8385-5dd5748b40aa", - "resource": { - "resourceType": "Observation", - "id": "76e59a72-a598-459f-8385-5dd5748b40aa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 10.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6214e182-c516-4c77-97ed-697c9670d15a", - "resource": { - "resourceType": "Observation", - "id": "6214e182-c516-4c77-97ed-697c9670d15a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 34.919, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9cec71c5-ca38-43b0-8a66-90e57aaf74fe", - "resource": { - "resourceType": "Observation", - "id": "9cec71c5-ca38-43b0-8a66-90e57aaf74fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 46.120, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:444065d3-2a85-4d0e-a068-3de5d0b4de9e", - "resource": { - "resourceType": "Observation", - "id": "444065d3-2a85-4d0e-a068-3de5d0b4de9e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2fabf9b3-2a7f-4a55-a521-5996546d5f99", - "resource": { - "resourceType": "Observation", - "id": "2fabf9b3-2a7f-4a55-a521-5996546d5f99", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 79, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ec2ca22f-bb9a-4a1f-96b8-de93d4e10a19", - "resource": { - "resourceType": "Observation", - "id": "ec2ca22f-bb9a-4a1f-96b8-de93d4e10a19", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7af7414f-6f62-455b-88a8-c3982090522f", - "resource": { - "resourceType": "Observation", - "id": "7af7414f-6f62-455b-88a8-c3982090522f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "effectiveDateTime": "2015-11-06T15:21:32-08:00", - "issued": "2015-11-06T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9bc202b-1079-4d36-898e-5731fa54fcfa", - "resource": { - "resourceType": "Immunization", - "id": "c9bc202b-1079-4d36-898e-5731fa54fcfa", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - }, - "date": "2015-11-06T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a55be85b-839f-44af-b1ae-bec185400bba", - "resource": { - "resourceType": "Claim", - "id": "a55be85b-839f-44af-b1ae-bec185400bba", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2015-11-06T15:21:32-08:00", - "end": "2015-11-06T15:36:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c9bc202b-1079-4d36-898e-5731fa54fcfa" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:02276df4-3e3e-44ff-87ae-4151d6cfdc6e" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b1dce11f-7e61-4fab-b324-1570b32e0cc9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b1dce11f-7e61-4fab-b324-1570b32e0cc9", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a55be85b-839f-44af-b1ae-bec185400bba" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-11-06T15:36:32-08:00", - "end": "2016-11-06T15:36:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c", - "resource": { - "resourceType": "Encounter", - "id": "bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2016-05-06T16:21:32-07:00", - "end": "2016-05-06T16:51:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4a15ce04-4d5e-40b4-b021-e875a315c38d", - "resource": { - "resourceType": "Observation", - "id": "4a15ce04-4d5e-40b4-b021-e875a315c38d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 86.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e6b135a-62c1-40f0-8b2d-1b5b93ce9544", - "resource": { - "resourceType": "Observation", - "id": "4e6b135a-62c1-40f0-8b2d-1b5b93ce9544", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:efc22b1e-f5eb-41da-9a21-8f807beaa716", - "resource": { - "resourceType": "Observation", - "id": "efc22b1e-f5eb-41da-9a21-8f807beaa716", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 12.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4df3a747-e642-49da-81c8-03e0aff973b5", - "resource": { - "resourceType": "Observation", - "id": "4df3a747-e642-49da-81c8-03e0aff973b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 53.504, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82cb153b-243e-4956-9bc1-1dab9c77aaab", - "resource": { - "resourceType": "Observation", - "id": "82cb153b-243e-4956-9bc1-1dab9c77aaab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 46.730, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:04b39abd-bb56-49da-9b14-acdc84b108d0", - "resource": { - "resourceType": "Observation", - "id": "04b39abd-bb56-49da-9b14-acdc84b108d0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 16.320, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f3c0bda-f6eb-40e5-98ac-696dd4158921", - "resource": { - "resourceType": "Observation", - "id": "1f3c0bda-f6eb-40e5-98ac-696dd4158921", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 57.496, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a63694d8-2566-4b9b-9af1-a416de6af074", - "resource": { - "resourceType": "Observation", - "id": "a63694d8-2566-4b9b-9af1-a416de6af074", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 105, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:015eb094-4fe7-4f44-bc06-80f09399a807", - "resource": { - "resourceType": "Observation", - "id": "015eb094-4fe7-4f44-bc06-80f09399a807", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 64, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db193a6c-a16d-4bf7-a2ae-42000e0d85bb", - "resource": { - "resourceType": "Observation", - "id": "db193a6c-a16d-4bf7-a2ae-42000e0d85bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cca53e11-8465-435e-b17d-8304bc48b70c", - "resource": { - "resourceType": "Observation", - "id": "cca53e11-8465-435e-b17d-8304bc48b70c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "effectiveDateTime": "2016-05-06T16:21:32-07:00", - "issued": "2016-05-06T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:31252f5d-b0ec-4505-b256-2507d4f838e1", - "resource": { - "resourceType": "Procedure", - "id": "31252f5d-b0ec-4505-b256-2507d4f838e1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "performedPeriod": { - "start": "2016-05-06T16:21:32-07:00", - "end": "2016-05-06T16:36:32-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:6480a0d3-7dcd-45f4-ae49-2e274b6b3877", - "resource": { - "resourceType": "Immunization", - "id": "6480a0d3-7dcd-45f4-ae49-2e274b6b3877", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - }, - "date": "2016-05-06T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a8c4cb23-c849-4b5a-a147-cbf51a070d7d", - "resource": { - "resourceType": "Claim", - "id": "a8c4cb23-c849-4b5a-a147-cbf51a070d7d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2016-05-06T16:21:32-07:00", - "end": "2016-05-06T16:51:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:6480a0d3-7dcd-45f4-ae49-2e274b6b3877" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:31252f5d-b0ec-4505-b256-2507d4f838e1" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:bd010c6a-8d1e-4a96-9caf-1b5d2ad2af6c" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 533.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:98b3240c-d25c-4191-a01e-003c3d600f00", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "98b3240c-d25c-4191-a01e-003c3d600f00", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a8c4cb23-c849-4b5a-a147-cbf51a070d7d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-05-06T16:51:32-07:00", - "end": "2017-05-06T16:51:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 106.702, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 426.808, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 533.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 533.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 539.224, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1", - "resource": { - "resourceType": "Encounter", - "id": "3748df54-cdc9-4672-b442-491ebec3dbd1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2016-11-04T16:21:32-07:00", - "end": "2016-11-04T16:51:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bcafb02d-bd3e-4bd0-806e-16175c4b1ca1", - "resource": { - "resourceType": "Observation", - "id": "bcafb02d-bd3e-4bd0-806e-16175c4b1ca1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 90.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2aacc861-4f5b-4ef0-bd90-d7674826b8cf", - "resource": { - "resourceType": "Observation", - "id": "2aacc861-4f5b-4ef0-bd90-d7674826b8cf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:049730f9-5c82-4073-b2a7-35c74f960006", - "resource": { - "resourceType": "Observation", - "id": "049730f9-5c82-4073-b2a7-35c74f960006", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 13.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:45ed0f26-69a5-4144-a4c3-017512f7d4ec", - "resource": { - "resourceType": "Observation", - "id": "45ed0f26-69a5-4144-a4c3-017512f7d4ec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 67.727, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e64b70ef-1c97-41cb-a9c5-92362a37b8ca", - "resource": { - "resourceType": "Observation", - "id": "e64b70ef-1c97-41cb-a9c5-92362a37b8ca", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 47.150, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68ca7411-c0ff-4d06-b123-3202a7b9ca12", - "resource": { - "resourceType": "Observation", - "id": "68ca7411-c0ff-4d06-b123-3202a7b9ca12", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 16.520, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9221b296-56c5-4a61-8802-6cacf107e5ac", - "resource": { - "resourceType": "Observation", - "id": "9221b296-56c5-4a61-8802-6cacf107e5ac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 71.965, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b185f5c-01f5-40c1-a1d7-bbf4edb054ac", - "resource": { - "resourceType": "Observation", - "id": "6b185f5c-01f5-40c1-a1d7-bbf4edb054ac", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:695086c1-30a2-4c0f-9ec0-5d03e2280ee6", - "resource": { - "resourceType": "Observation", - "id": "695086c1-30a2-4c0f-9ec0-5d03e2280ee6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3eb1df1b-03c9-41dd-9ab4-92233babedd2", - "resource": { - "resourceType": "Observation", - "id": "3eb1df1b-03c9-41dd-9ab4-92233babedd2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6b133cbf-5aea-4a88-babb-36140361eb73", - "resource": { - "resourceType": "Observation", - "id": "6b133cbf-5aea-4a88-babb-36140361eb73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "effectiveDateTime": "2016-11-04T16:21:32-07:00", - "issued": "2016-11-04T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc05be7e-a85d-4dd9-828f-608cd633dc06", - "resource": { - "resourceType": "Procedure", - "id": "dc05be7e-a85d-4dd9-828f-608cd633dc06", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "performedPeriod": { - "start": "2016-11-04T16:21:32-07:00", - "end": "2016-11-04T16:36:32-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:dd99667c-3321-4113-90c1-7f3a98eb7df8", - "resource": { - "resourceType": "Immunization", - "id": "dd99667c-3321-4113-90c1-7f3a98eb7df8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - }, - "date": "2016-11-04T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7e3ae344-23f3-4e7e-97ae-f475dd8c8d70", - "resource": { - "resourceType": "Claim", - "id": "7e3ae344-23f3-4e7e-97ae-f475dd8c8d70", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2016-11-04T16:21:32-07:00", - "end": "2016-11-04T16:51:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:dd99667c-3321-4113-90c1-7f3a98eb7df8" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:dc05be7e-a85d-4dd9-828f-608cd633dc06" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3748df54-cdc9-4672-b442-491ebec3dbd1" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 590.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:83ec271b-be55-4234-893b-05e07574e11a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "83ec271b-be55-4234-893b-05e07574e11a", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7e3ae344-23f3-4e7e-97ae-f475dd8c8d70" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-11-04T16:51:32-07:00", - "end": "2017-11-04T16:51:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 118.184, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 472.736, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 590.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 590.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 585.152, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53", - "resource": { - "resourceType": "Encounter", - "id": "bcc723ef-94f8-4e94-8d33-cc6ae15f1e53", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2017-05-05T16:21:32-07:00", - "end": "2017-05-05T16:36:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3d7c95a1-205f-4ff1-a65c-58f1f52f5669", - "resource": { - "resourceType": "Observation", - "id": "3d7c95a1-205f-4ff1-a65c-58f1f52f5669", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 93.300, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5bee4bf-65bd-461c-9f2f-13d9423d5146", - "resource": { - "resourceType": "Observation", - "id": "f5bee4bf-65bd-461c-9f2f-13d9423d5146", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8375b59-43b1-4a84-9720-1e5b6fa73ff2", - "resource": { - "resourceType": "Observation", - "id": "b8375b59-43b1-4a84-9720-1e5b6fa73ff2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 14.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7284dfb-e35e-4da9-a54b-cfe4d5ea3b7c", - "resource": { - "resourceType": "Observation", - "id": "b7284dfb-e35e-4da9-a54b-cfe4d5ea3b7c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 66.436, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:806ddaa6-b8f1-4935-bf6c-4108430ef673", - "resource": { - "resourceType": "Observation", - "id": "806ddaa6-b8f1-4935-bf6c-4108430ef673", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 47.180, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:61819254-bd76-4a89-b804-bdc49b7c6baf", - "resource": { - "resourceType": "Observation", - "id": "61819254-bd76-4a89-b804-bdc49b7c6baf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 16.230, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7256b2e-87b9-4b62-8e63-9073fb478327", - "resource": { - "resourceType": "Observation", - "id": "a7256b2e-87b9-4b62-8e63-9073fb478327", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 71.086, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a6db32f1-e01f-4faa-986b-e09986deb3e3", - "resource": { - "resourceType": "Observation", - "id": "a6db32f1-e01f-4faa-986b-e09986deb3e3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 124, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:96c055e8-f34e-4b0e-af40-c2a5f4430534", - "resource": { - "resourceType": "Observation", - "id": "96c055e8-f34e-4b0e-af40-c2a5f4430534", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e9b59076-75e2-4f17-bbad-67adfc435488", - "resource": { - "resourceType": "Observation", - "id": "e9b59076-75e2-4f17-bbad-67adfc435488", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f1a2600c-f975-4da3-86cb-ad17fb7b9d6b", - "resource": { - "resourceType": "Observation", - "id": "f1a2600c-f975-4da3-86cb-ad17fb7b9d6b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - }, - "effectiveDateTime": "2017-05-05T16:21:32-07:00", - "issued": "2017-05-05T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fa5a7301-20e3-450d-b0ce-e0bfdc1bb814", - "resource": { - "resourceType": "Claim", - "id": "fa5a7301-20e3-450d-b0ce-e0bfdc1bb814", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2017-05-05T16:21:32-07:00", - "end": "2017-05-05T16:36:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:bcc723ef-94f8-4e94-8d33-cc6ae15f1e53" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f204debd-3e15-49cd-a0eb-45274af5d0be", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f204debd-3e15-49cd-a0eb-45274af5d0be", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "fa5a7301-20e3-450d-b0ce-e0bfdc1bb814" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2017-05-05T16:36:32-07:00", - "end": "2018-05-05T16:36:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b", - "resource": { - "resourceType": "Encounter", - "id": "d2763c1d-b19b-47bb-b734-9c49421c839b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2017-11-03T16:21:32-07:00", - "end": "2017-11-03T16:36:32-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:45f5a054-e562-4041-851d-f50a637a57fc", - "resource": { - "resourceType": "Observation", - "id": "45f5a054-e562-4041-851d-f50a637a57fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 96.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0895469c-81e2-41ec-944e-563cc2cd9022", - "resource": { - "resourceType": "Observation", - "id": "0895469c-81e2-41ec-944e-563cc2cd9022", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd9e7976-9c07-43b8-8532-b613c74522f9", - "resource": { - "resourceType": "Observation", - "id": "cd9e7976-9c07-43b8-8532-b613c74522f9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 14.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:293885c5-daa6-49f1-a66b-46b854ccec7f", - "resource": { - "resourceType": "Observation", - "id": "293885c5-daa6-49f1-a66b-46b854ccec7f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 66.436, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30b28a63-4ffa-49b9-a1c5-35764d2bc5b1", - "resource": { - "resourceType": "Observation", - "id": "30b28a63-4ffa-49b9-a1c5-35764d2bc5b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 47.180, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cc527b5-99ca-492c-97df-d1f5b0b373d5", - "resource": { - "resourceType": "Observation", - "id": "0cc527b5-99ca-492c-97df-d1f5b0b373d5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 15.890, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc1b1af5-9d52-4828-aa62-279a5f48a085", - "resource": { - "resourceType": "Observation", - "id": "fc1b1af5-9d52-4828-aa62-279a5f48a085", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 67.013, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8de69a0f-6463-4e45-a347-b8acb729ff60", - "resource": { - "resourceType": "Observation", - "id": "8de69a0f-6463-4e45-a347-b8acb729ff60", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 119, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd2d183c-53ff-427a-9ca9-9ccfbd2bab07", - "resource": { - "resourceType": "Observation", - "id": "dd2d183c-53ff-427a-9ca9-9ccfbd2bab07", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 64, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3fcf856f-ead4-45a4-b338-4dcafb1bb80e", - "resource": { - "resourceType": "Observation", - "id": "3fcf856f-ead4-45a4-b338-4dcafb1bb80e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:985cb719-8660-4129-9eec-f4b4cf9983b0", - "resource": { - "resourceType": "Observation", - "id": "985cb719-8660-4129-9eec-f4b4cf9983b0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "effectiveDateTime": "2017-11-03T16:21:32-07:00", - "issued": "2017-11-03T16:21:32.447-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed6030da-081e-4ff2-9765-ea61f3c734f4", - "resource": { - "resourceType": "Immunization", - "id": "ed6030da-081e-4ff2-9765-ea61f3c734f4", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - }, - "date": "2017-11-03T16:21:32-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:266dfe29-12d3-4744-a862-f2f1c8aa33e0", - "resource": { - "resourceType": "Claim", - "id": "266dfe29-12d3-4744-a862-f2f1c8aa33e0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2017-11-03T16:21:32-07:00", - "end": "2017-11-03T16:36:32-07:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ed6030da-081e-4ff2-9765-ea61f3c734f4" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:d2763c1d-b19b-47bb-b734-9c49421c839b" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4d408cf9-09d1-4007-bc07-76cbaefda1c1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4d408cf9-09d1-4007-bc07-76cbaefda1c1", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "266dfe29-12d3-4744-a862-f2f1c8aa33e0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2017-11-03T16:36:32-07:00", - "end": "2018-11-03T16:36:32-07:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b", - "resource": { - "resourceType": "Encounter", - "id": "380edf2f-79ab-481f-92fa-153d501d5e3b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2018-11-09T15:21:32-08:00", - "end": "2018-11-09T15:51:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:96d199a3-5b05-4893-8261-4c20493535dc", - "resource": { - "resourceType": "Observation", - "id": "96d199a3-5b05-4893-8261-4c20493535dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 103.10, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:304e7a9e-9ab0-4df9-8a03-feac0be792fc", - "resource": { - "resourceType": "Observation", - "id": "304e7a9e-9ab0-4df9-8a03-feac0be792fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ffca0d8d-0f87-4004-93ba-553f21e25709", - "resource": { - "resourceType": "Observation", - "id": "ffca0d8d-0f87-4004-93ba-553f21e25709", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 16.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:003ebfe2-cd4a-4c2a-bbd3-732a9eb1343a", - "resource": { - "resourceType": "Observation", - "id": "003ebfe2-cd4a-4c2a-bbd3-732a9eb1343a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 15.100, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f90e75b2-44b6-4497-9a72-58137a1deb39", - "resource": { - "resourceType": "Observation", - "id": "f90e75b2-44b6-4497-9a72-58137a1deb39", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 48.190, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7d08d1fe-e937-43d4-a470-3dccb2eea9b5", - "resource": { - "resourceType": "Observation", - "id": "7d08d1fe-e937-43d4-a470-3dccb2eea9b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2f3393b3-b75d-443f-932e-6f216fe6bcdb", - "resource": { - "resourceType": "Observation", - "id": "2f3393b3-b75d-443f-932e-6f216fe6bcdb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 86, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49e6af52-bf0c-4766-9060-bebc09efd71a", - "resource": { - "resourceType": "Observation", - "id": "49e6af52-bf0c-4766-9060-bebc09efd71a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8315e0fb-ac6c-4b85-99e8-0c09f32b5bd3", - "resource": { - "resourceType": "Observation", - "id": "8315e0fb-ac6c-4b85-99e8-0c09f32b5bd3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "effectiveDateTime": "2018-11-09T15:21:32-08:00", - "issued": "2018-11-09T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c706176f-4627-4e21-bb9d-75785b155ed9", - "resource": { - "resourceType": "Procedure", - "id": "c706176f-4627-4e21-bb9d-75785b155ed9", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "performedPeriod": { - "start": "2018-11-09T15:21:32-08:00", - "end": "2018-11-09T15:36:32-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c75c5718-8596-40ed-b9d4-3e2cad66bcdd", - "resource": { - "resourceType": "Immunization", - "id": "c75c5718-8596-40ed-b9d4-3e2cad66bcdd", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "date": "2018-11-09T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:56761bf9-a4e5-400f-a96f-7426ef48e3c1", - "resource": { - "resourceType": "Immunization", - "id": "56761bf9-a4e5-400f-a96f-7426ef48e3c1", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "date": "2018-11-09T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0066d618-88a8-4eaf-b66d-9600eb6608e6", - "resource": { - "resourceType": "Immunization", - "id": "0066d618-88a8-4eaf-b66d-9600eb6608e6", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "date": "2018-11-09T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b6c3ef13-394b-4131-92dd-df2a28ddb4df", - "resource": { - "resourceType": "Immunization", - "id": "b6c3ef13-394b-4131-92dd-df2a28ddb4df", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "date": "2018-11-09T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4e3b4c39-cb94-4c09-b2db-127a25e80426", - "resource": { - "resourceType": "Immunization", - "id": "4e3b4c39-cb94-4c09-b2db-127a25e80426", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - }, - "date": "2018-11-09T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f5173492-5912-4fd2-aa9b-dcf60d6709dd", - "resource": { - "resourceType": "Claim", - "id": "f5173492-5912-4fd2-aa9b-dcf60d6709dd", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2018-11-09T15:21:32-08:00", - "end": "2018-11-09T15:51:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c75c5718-8596-40ed-b9d4-3e2cad66bcdd" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:56761bf9-a4e5-400f-a96f-7426ef48e3c1" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:0066d618-88a8-4eaf-b66d-9600eb6608e6" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:b6c3ef13-394b-4131-92dd-df2a28ddb4df" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4e3b4c39-cb94-4c09-b2db-127a25e80426" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:c706176f-4627-4e21-bb9d-75785b155ed9" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:380edf2f-79ab-481f-92fa-153d501d5e3b" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 338.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:aba58129-01b1-459e-87e9-21e505c1cba7", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "aba58129-01b1-459e-87e9-21e505c1cba7", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f5173492-5912-4fd2-aa9b-dcf60d6709dd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-11-09T15:51:32-08:00", - "end": "2019-11-09T15:51:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 67.68400000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 270.73600000000005, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 338.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 338.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 832.816, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504", - "resource": { - "resourceType": "Encounter", - "id": "5bea4746-76ab-4b50-b09d-3337227f8504", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - } - ], - "period": { - "start": "2019-11-15T15:21:32-08:00", - "end": "2019-11-15T15:36:32-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9b2ade99-39f7-4d8c-a92a-3744c38ca0c9", - "resource": { - "resourceType": "Observation", - "id": "9b2ade99-39f7-4d8c-a92a-3744c38ca0c9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 109.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5146147e-2697-4175-8d81-0218ff1ea6e5", - "resource": { - "resourceType": "Observation", - "id": "5146147e-2697-4175-8d81-0218ff1ea6e5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:267e9107-1ad0-4185-8ceb-74fa5a1f047f", - "resource": { - "resourceType": "Observation", - "id": "267e9107-1ad0-4185-8ceb-74fa5a1f047f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 19.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ccca442-c229-4689-bd35-12420ed9c3aa", - "resource": { - "resourceType": "Observation", - "id": "1ccca442-c229-4689-bd35-12420ed9c3aa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 16.230, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e05d098e-a8f2-4a73-a505-512458a346c4", - "resource": { - "resourceType": "Observation", - "id": "e05d098e-a8f2-4a73-a505-512458a346c4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 73.726, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e3f0218-3535-4659-8bb5-114d70255c19", - "resource": { - "resourceType": "Observation", - "id": "6e3f0218-3535-4659-8bb5-114d70255c19", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eab9a943-9467-4d03-bd7f-513d1f93520d", - "resource": { - "resourceType": "Observation", - "id": "eab9a943-9467-4d03-bd7f-513d1f93520d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 88, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7fe674c8-901c-4de2-bc70-5098bb8b9ca5", - "resource": { - "resourceType": "Observation", - "id": "7fe674c8-901c-4de2-bc70-5098bb8b9ca5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3498e489-4ac0-4496-93a1-b765716a6f11", - "resource": { - "resourceType": "Observation", - "id": "3498e489-4ac0-4496-93a1-b765716a6f11", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 9.7329, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:65318aa0-6b55-46d2-880b-b188d5459af3", - "resource": { - "resourceType": "Observation", - "id": "65318aa0-6b55-46d2-880b-b188d5459af3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 5.3382, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", - "resource": { - "resourceType": "Observation", - "id": "bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 16.992, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", - "resource": { - "resourceType": "Observation", - "id": "4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 37.183, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b687de9-a614-481c-ba0d-f1fcf298045d", - "resource": { - "resourceType": "Observation", - "id": "0b687de9-a614-481c-ba0d-f1fcf298045d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 82.592, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b52e4ee4-a6e1-466e-a053-544f494b359c", - "resource": { - "resourceType": "Observation", - "id": "b52e4ee4-a6e1-466e-a053-544f494b359c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 30.831, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:79883a19-1cf1-4e83-ac1a-0986a6e4aed4", - "resource": { - "resourceType": "Observation", - "id": "79883a19-1cf1-4e83-ac1a-0986a6e4aed4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 35.670, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e821119-0284-4061-9305-42a9fbc53df8", - "resource": { - "resourceType": "Observation", - "id": "0e821119-0284-4061-9305-42a9fbc53df8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 42.759, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f528b8c2-be75-4361-8f81-9f2e92c5b337", - "resource": { - "resourceType": "Observation", - "id": "f528b8c2-be75-4361-8f81-9f2e92c5b337", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 428.98, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e021bd53-bf27-439c-a94a-ba9272004957", - "resource": { - "resourceType": "Observation", - "id": "e021bd53-bf27-439c-a94a-ba9272004957", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 280.42, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1295d24b-1ed4-46f1-9f2e-b138413b7348", - "resource": { - "resourceType": "Observation", - "id": "1295d24b-1ed4-46f1-9f2e-b138413b7348", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueQuantity": { - "value": 9.4251, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd782321-ba34-4452-a575-782f2448b7b8", - "resource": { - "resourceType": "Observation", - "id": "dd782321-ba34-4452-a575-782f2448b7b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7425a267-f35e-4ab8-a6bb-38c16ca67268", - "resource": { - "resourceType": "Immunization", - "id": "7425a267-f35e-4ab8-a6bb-38c16ca67268", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "encounter": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "date": "2019-11-15T15:21:32-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8eae1649-e850-4c94-8ed1-bbb619dcafe8", - "resource": { - "resourceType": "DiagnosticReport", - "id": "8eae1649-e850-4c94-8ed1-bbb619dcafe8", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "context": { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - }, - "effectiveDateTime": "2019-11-15T15:21:32-08:00", - "issued": "2019-11-15T15:21:32.447-08:00", - "result": [ - { - "reference": "urn:uuid:3498e489-4ac0-4496-93a1-b765716a6f11", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:65318aa0-6b55-46d2-880b-b188d5459af3", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:bbf1003e-c1d2-48c9-95f9-023e31fd5ec2", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:4ae9a4d4-9cf5-41a5-b3cb-3d33d8f1384c", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:0b687de9-a614-481c-ba0d-f1fcf298045d", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:b52e4ee4-a6e1-466e-a053-544f494b359c", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:79883a19-1cf1-4e83-ac1a-0986a6e4aed4", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:0e821119-0284-4061-9305-42a9fbc53df8", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:f528b8c2-be75-4361-8f81-9f2e92c5b337", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:e021bd53-bf27-439c-a94a-ba9272004957", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1295d24b-1ed4-46f1-9f2e-b138413b7348", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:dd5b29e0-824a-4272-8d8b-9649c75edb0c", - "resource": { - "resourceType": "Claim", - "id": "dd5b29e0-824a-4272-8d8b-9649c75edb0c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "start": "2019-11-15T15:21:32-08:00", - "end": "2019-11-15T15:36:32-08:00" - }, - "organization": { - "reference": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7425a267-f35e-4ab8-a6bb-38c16ca67268" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5bea4746-76ab-4b50-b09d-3337227f8504" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ecba9532-715b-4d57-816a-3f0e00f9a67c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ecba9532-715b-4d57-816a-3f0e00f9a67c", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Anthem" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "dd5b29e0-824a-4272-8d8b-9649c75edb0c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:df27a976-5c5e-4b84-ad00-fe32972dce9c" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-11-15T15:36:32-08:00", - "end": "2020-11-15T15:36:32-08:00" - }, - "provider": { - "identifier": { - "value": "f9f09686-21e4-3b27-ba68-9b7313d9a553" - } - }, - "organization": { - "identifier": { - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json deleted file mode 100644 index 65e224b7d394..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Damon455_Will178_136f997d-0a94-4573-97da-a53b5060a612.json +++ /dev/null @@ -1,28126 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612", - "resource": { - "resourceType": "Patient", - "id": "136f997d-0a94-4573-97da-a53b5060a612", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: -3753024713238705289 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2028-9", - "display": "Asian" - } - }, - { - "url": "text", - "valueString": "Asian" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Fredericka800 Bayer639" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "M" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Hatfield", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.002142737663232659 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 6.997857262336767 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "136f997d-0a94-4573-97da-a53b5060a612" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "136f997d-0a94-4573-97da-a53b5060a612" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-87-6648" - } - ], - "name": [ - { - "use": "official", - "family": "Will178", - "given": [ - "Damon455" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-923-1711", - "use": "home" - } - ], - "gender": "male", - "birthDate": "2012-01-04", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.360699590968395 - }, - { - "url": "longitude", - "valueDecimal": -71.15833176606733 - } - ] - } - ], - "line": [ - "280 Lesch Junction" - ], - "city": "Cambridge", - "state": "Massachusetts", - "postalCode": "02140", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245", - "resource": { - "resourceType": "Organization", - "id": "9bd6ea26-d344-3409-aea7-2e925fd0d245", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP32892", - "telecom": [ - { - "system": "phone", - "value": "617-744-6527" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.369451 - }, - { - "url": "longitude", - "valueDecimal": -71.177925 - } - ] - } - ], - "line": [ - "158 A BELMONT ST" - ], - "city": "WATERTOWN", - "state": "MA", - "postalCode": "02472-2912", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0", - "resource": { - "resourceType": "Practitioner", - "id": "ba314750-701e-3370-ade0-270e178d80e0", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "22240" - } - ], - "active": true, - "name": [ - { - "family": "Pacocha935", - "given": [ - "Ula130" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "158 A BELMONT ST" - ], - "city": "WATERTOWN", - "state": "MA", - "postalCode": "02472-2912", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350", - "resource": { - "resourceType": "Encounter", - "id": "aa114ab8-742d-4737-88a5-1e216870f350", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-01-04T20:05:11-08:00", - "end": "2012-01-04T20:35:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b5264b2e-b210-4b4f-954b-96ab336f1e85", - "resource": { - "resourceType": "Observation", - "id": "b5264b2e-b210-4b4f-954b-96ab336f1e85", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 50.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:829ed4fa-5bec-4d81-acf6-053d0970e678", - "resource": { - "resourceType": "Observation", - "id": "829ed4fa-5bec-4d81-acf6-053d0970e678", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7adaafd6-d242-4d5c-8dcc-45cdc47773f2", - "resource": { - "resourceType": "Observation", - "id": "7adaafd6-d242-4d5c-8dcc-45cdc47773f2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 3.6000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:26e5ad01-5e59-48a0-8e09-7bbe2561f586", - "resource": { - "resourceType": "Observation", - "id": "26e5ad01-5e59-48a0-8e09-7bbe2561f586", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 37.964, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:324535ed-c3cd-4341-baf0-f55c9d76f5e4", - "resource": { - "resourceType": "Observation", - "id": "324535ed-c3cd-4341-baf0-f55c9d76f5e4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 34.180, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:faf8f54d-ecdb-44be-9a56-12d8462f680d", - "resource": { - "resourceType": "Observation", - "id": "faf8f54d-ecdb-44be-9a56-12d8462f680d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:30a7cdef-02ed-46d0-93fa-adacabc8ef98", - "resource": { - "resourceType": "Observation", - "id": "30a7cdef-02ed-46d0-93fa-adacabc8ef98", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 84, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d7e9211-6989-437a-9573-5a631cc52c7a", - "resource": { - "resourceType": "Observation", - "id": "6d7e9211-6989-437a-9573-5a631cc52c7a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", - "resource": { - "resourceType": "Observation", - "id": "ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 6.5440, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3953521d-dced-4d33-9bd7-60c1b3147e6c", - "resource": { - "resourceType": "Observation", - "id": "3953521d-dced-4d33-9bd7-60c1b3147e6c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 5.1374, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f77c37e4-57de-4027-a54d-aa02092f5011", - "resource": { - "resourceType": "Observation", - "id": "f77c37e4-57de-4027-a54d-aa02092f5011", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 16.950, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5797b11d-116a-4a10-92a8-54a137971870", - "resource": { - "resourceType": "Observation", - "id": "5797b11d-116a-4a10-92a8-54a137971870", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 39.246, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:658ad303-82c7-455f-b365-5acb63a8ca74", - "resource": { - "resourceType": "Observation", - "id": "658ad303-82c7-455f-b365-5acb63a8ca74", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 88.304, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:da416a2c-e670-4a06-8197-db911e5213ab", - "resource": { - "resourceType": "Observation", - "id": "da416a2c-e670-4a06-8197-db911e5213ab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 27.571, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", - "resource": { - "resourceType": "Observation", - "id": "2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 33.153, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:365e9ab9-da06-4c1b-a628-7fdb7279e08d", - "resource": { - "resourceType": "Observation", - "id": "365e9ab9-da06-4c1b-a628-7fdb7279e08d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 44.607, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", - "resource": { - "resourceType": "Observation", - "id": "23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 238.70, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aa2c3dad-a1cc-4319-882e-4f270e9448f5", - "resource": { - "resourceType": "Observation", - "id": "aa2c3dad-a1cc-4319-882e-4f270e9448f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 267.69, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29fd3111-9833-4d16-b742-9a88d528958e", - "resource": { - "resourceType": "Observation", - "id": "29fd3111-9833-4d16-b742-9a88d528958e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 11.169, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4d5bd8fc-7cb9-4ca8-b2bb-dcb9a2bf8b69", - "resource": { - "resourceType": "Observation", - "id": "4d5bd8fc-7cb9-4ca8-b2bb-dcb9a2bf8b69", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50116840-3db7-432d-9b08-cd549b005bbf", - "resource": { - "resourceType": "Procedure", - "id": "50116840-3db7-432d-9b08-cd549b005bbf", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "performedPeriod": { - "start": "2012-01-04T20:05:11-08:00", - "end": "2012-01-04T20:20:11-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:947f8511-77e9-4208-b330-56896f4ca20a", - "resource": { - "resourceType": "Immunization", - "id": "947f8511-77e9-4208-b330-56896f4ca20a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "date": "2012-01-04T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:6673c56f-3907-419c-bf9a-cc07efb1e250", - "resource": { - "resourceType": "DiagnosticReport", - "id": "6673c56f-3907-419c-bf9a-cc07efb1e250", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - }, - "effectiveDateTime": "2012-01-04T20:05:11-08:00", - "issued": "2012-01-04T20:05:11.927-08:00", - "result": [ - { - "reference": "urn:uuid:ff5c5fcc-6b2d-4d31-882f-519c97bbdb6a", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:3953521d-dced-4d33-9bd7-60c1b3147e6c", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:f77c37e4-57de-4027-a54d-aa02092f5011", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:5797b11d-116a-4a10-92a8-54a137971870", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:658ad303-82c7-455f-b365-5acb63a8ca74", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:da416a2c-e670-4a06-8197-db911e5213ab", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:2313aafb-5a82-4e9d-bf75-1f9d2ad361a0", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:365e9ab9-da06-4c1b-a628-7fdb7279e08d", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:23ccbdc4-73c7-43bd-8c99-d0d57cec04a1", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:aa2c3dad-a1cc-4319-882e-4f270e9448f5", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:29fd3111-9833-4d16-b742-9a88d528958e", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:5e317710-ebf3-4e4d-839b-91dd57dabd65", - "resource": { - "resourceType": "Claim", - "id": "5e317710-ebf3-4e4d-839b-91dd57dabd65", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-01-04T20:05:11-08:00", - "end": "2012-01-04T20:35:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:947f8511-77e9-4208-b330-56896f4ca20a" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:50116840-3db7-432d-9b08-cd549b005bbf" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:aa114ab8-742d-4737-88a5-1e216870f350" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 865.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:be632d75-e4a3-43ed-ac34-00dad401932e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "be632d75-e4a3-43ed-ac34-00dad401932e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5e317710-ebf3-4e4d-839b-91dd57dabd65" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-01-04T20:35:11-08:00", - "end": "2013-01-04T20:35:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 173.074, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 692.296, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 865.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 865.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 804.7120000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462", - "resource": { - "resourceType": "Encounter", - "id": "61e62625-d724-4f99-9172-5447a5868462", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-02-08T20:05:11-08:00", - "end": "2012-02-08T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:aa656ec3-26ce-4121-a8ef-11e14743b047", - "resource": { - "resourceType": "Observation", - "id": "aa656ec3-26ce-4121-a8ef-11e14743b047", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 54.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:71b88e97-35df-4a1f-bb76-326de8e5c030", - "resource": { - "resourceType": "Observation", - "id": "71b88e97-35df-4a1f-bb76-326de8e5c030", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1332afae-b398-4aa9-8c43-8d0e3de12470", - "resource": { - "resourceType": "Observation", - "id": "1332afae-b398-4aa9-8c43-8d0e3de12470", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 4.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d0772d3-b016-455f-aae8-61e18db764eb", - "resource": { - "resourceType": "Observation", - "id": "6d0772d3-b016-455f-aae8-61e18db764eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 21.966, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eeb7ad30-699c-4a2b-b402-a354bbf48ad2", - "resource": { - "resourceType": "Observation", - "id": "eeb7ad30-699c-4a2b-b402-a354bbf48ad2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 37.850, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:990ad540-ad95-4d8d-aac6-4a7fd2df2811", - "resource": { - "resourceType": "Observation", - "id": "990ad540-ad95-4d8d-aac6-4a7fd2df2811", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2303615-c9da-4937-81fe-a9403e06ea01", - "resource": { - "resourceType": "Observation", - "id": "b2303615-c9da-4937-81fe-a9403e06ea01", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 63, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed840bf4-fa3a-4570-8cf4-9d58bbfb99c2", - "resource": { - "resourceType": "Observation", - "id": "ed840bf4-fa3a-4570-8cf4-9d58bbfb99c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0f3a9ab-5068-4646-9c8c-2d08477b529c", - "resource": { - "resourceType": "Observation", - "id": "b0f3a9ab-5068-4646-9c8c-2d08477b529c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "effectiveDateTime": "2012-02-08T20:05:11-08:00", - "issued": "2012-02-08T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d842fbbd-0a13-4d65-a515-5d85b0831959", - "resource": { - "resourceType": "Immunization", - "id": "d842fbbd-0a13-4d65-a515-5d85b0831959", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - }, - "date": "2012-02-08T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:6163cc16-6018-461f-9cc7-63c4dc2c75f5", - "resource": { - "resourceType": "Claim", - "id": "6163cc16-6018-461f-9cc7-63c4dc2c75f5", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-02-08T20:05:11-08:00", - "end": "2012-02-08T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:d842fbbd-0a13-4d65-a515-5d85b0831959" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:61e62625-d724-4f99-9172-5447a5868462" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4bf41bb7-51dd-4093-b96c-caad9726af3d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4bf41bb7-51dd-4093-b96c-caad9726af3d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6163cc16-6018-461f-9cc7-63c4dc2c75f5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-02-08T20:20:11-08:00", - "end": "2013-02-08T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766", - "resource": { - "resourceType": "Encounter", - "id": "64cbe9ca-4dc1-4fad-9ec3-d20955aef766", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-04-11T21:05:11-07:00", - "end": "2012-04-11T21:20:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:86bea6a6-416d-45e9-99c3-24cf7df16084", - "resource": { - "resourceType": "Observation", - "id": "86bea6a6-416d-45e9-99c3-24cf7df16084", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 60.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb9238b1-f746-4d46-b014-fc6e46a4f633", - "resource": { - "resourceType": "Observation", - "id": "cb9238b1-f746-4d46-b014-fc6e46a4f633", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:15c9a3c1-c246-46c2-959c-ce7f332d9c69", - "resource": { - "resourceType": "Observation", - "id": "15c9a3c1-c246-46c2-959c-ce7f332d9c69", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 5.9000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:240b9973-55f3-4cea-a66c-96b4a08e1eea", - "resource": { - "resourceType": "Observation", - "id": "240b9973-55f3-4cea-a66c-96b4a08e1eea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 40.884, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:debaa3ea-31cc-4cf1-93a7-314add05ad2a", - "resource": { - "resourceType": "Observation", - "id": "debaa3ea-31cc-4cf1-93a7-314add05ad2a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 40.580, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9a1904d6-d4c0-4077-9f58-ec7dcf74cc1d", - "resource": { - "resourceType": "Observation", - "id": "9a1904d6-d4c0-4077-9f58-ec7dcf74cc1d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 125, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:023bd975-eb45-4b07-9ee8-81ff5f32f708", - "resource": { - "resourceType": "Observation", - "id": "023bd975-eb45-4b07-9ee8-81ff5f32f708", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 70, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d0874b36-2a04-427c-ad73-ebe6f69bbd0a", - "resource": { - "resourceType": "Observation", - "id": "d0874b36-2a04-427c-ad73-ebe6f69bbd0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3459c9f8-4eed-44ae-89e0-62ea0770bc72", - "resource": { - "resourceType": "Observation", - "id": "3459c9f8-4eed-44ae-89e0-62ea0770bc72", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "effectiveDateTime": "2012-04-11T21:05:11-07:00", - "issued": "2012-04-11T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f060c57b-71d2-40b0-8f5d-3a2fafb6fd94", - "resource": { - "resourceType": "Immunization", - "id": "f060c57b-71d2-40b0-8f5d-3a2fafb6fd94", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "date": "2012-04-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:51b7dff3-cdb3-4041-a1b4-11ad92001833", - "resource": { - "resourceType": "Immunization", - "id": "51b7dff3-cdb3-4041-a1b4-11ad92001833", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "date": "2012-04-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c09e6b38-a2ac-46f5-9575-b65025314de7", - "resource": { - "resourceType": "Immunization", - "id": "c09e6b38-a2ac-46f5-9575-b65025314de7", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "date": "2012-04-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:14ddc70f-6c5e-4c36-a1f9-56f673275141", - "resource": { - "resourceType": "Immunization", - "id": "14ddc70f-6c5e-4c36-a1f9-56f673275141", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "date": "2012-04-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:3260d808-6e20-4e34-bca4-e34d08f54147", - "resource": { - "resourceType": "Immunization", - "id": "3260d808-6e20-4e34-bca4-e34d08f54147", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - }, - "date": "2012-04-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0431adb2-45f2-4c54-a6f9-59fb0d69cda6", - "resource": { - "resourceType": "Claim", - "id": "0431adb2-45f2-4c54-a6f9-59fb0d69cda6", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-04-11T21:05:11-07:00", - "end": "2012-04-11T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f060c57b-71d2-40b0-8f5d-3a2fafb6fd94" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:51b7dff3-cdb3-4041-a1b4-11ad92001833" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c09e6b38-a2ac-46f5-9575-b65025314de7" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:14ddc70f-6c5e-4c36-a1f9-56f673275141" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:3260d808-6e20-4e34-bca4-e34d08f54147" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:64cbe9ca-4dc1-4fad-9ec3-d20955aef766" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fb411ea0-7750-45af-9df4-51e68aea0a60", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "fb411ea0-7750-45af-9df4-51e68aea0a60", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0431adb2-45f2-4c54-a6f9-59fb0d69cda6" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-04-11T21:20:11-07:00", - "end": "2013-04-11T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890", - "resource": { - "resourceType": "Encounter", - "id": "2941c94b-b52e-4699-943c-1674670cf890", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-06-13T21:05:11-07:00", - "end": "2012-06-13T21:35:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a0c4b166-927a-4e09-a052-abe681ce9181", - "resource": { - "resourceType": "Observation", - "id": "a0c4b166-927a-4e09-a052-abe681ce9181", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 64.100, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c2668ad1-d11f-41c7-8afc-da4535cd4f4b", - "resource": { - "resourceType": "Observation", - "id": "c2668ad1-d11f-41c7-8afc-da4535cd4f4b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:49e55983-39ec-40b3-8855-4830f2edbf5a", - "resource": { - "resourceType": "Observation", - "id": "49e55983-39ec-40b3-8855-4830f2edbf5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 7.1000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e393ff85-49a0-4dd5-a9b8-af2cc7d9b11b", - "resource": { - "resourceType": "Observation", - "id": "e393ff85-49a0-4dd5-a9b8-af2cc7d9b11b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 53.430, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a100883-6942-4c35-a051-0a5e86babba0", - "resource": { - "resourceType": "Observation", - "id": "1a100883-6942-4c35-a051-0a5e86babba0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 42.310, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4302df08-3c6f-4b2b-9ec7-ef584b2c62ab", - "resource": { - "resourceType": "Observation", - "id": "4302df08-3c6f-4b2b-9ec7-ef584b2c62ab", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 132, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c5b5c5ee-3a41-45c4-b5dc-13c6606b5185", - "resource": { - "resourceType": "Observation", - "id": "c5b5c5ee-3a41-45c4-b5dc-13c6606b5185", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e181255b-23ff-49e5-b04a-1a2a7a8b88a3", - "resource": { - "resourceType": "Observation", - "id": "e181255b-23ff-49e5-b04a-1a2a7a8b88a3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f2f7cfe-1156-4616-88a6-38aa826f2857", - "resource": { - "resourceType": "Observation", - "id": "9f2f7cfe-1156-4616-88a6-38aa826f2857", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "effectiveDateTime": "2012-06-13T21:05:11-07:00", - "issued": "2012-06-13T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d38dd1e-5695-487c-b565-e685839e7149", - "resource": { - "resourceType": "Procedure", - "id": "6d38dd1e-5695-487c-b565-e685839e7149", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "performedPeriod": { - "start": "2012-06-13T21:05:11-07:00", - "end": "2012-06-13T21:20:11-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:44758e37-5794-4ca8-b022-1f11b0cbde52", - "resource": { - "resourceType": "Immunization", - "id": "44758e37-5794-4ca8-b022-1f11b0cbde52", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "date": "2012-06-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:21a76df9-25e4-442a-ad18-81a22f061db8", - "resource": { - "resourceType": "Immunization", - "id": "21a76df9-25e4-442a-ad18-81a22f061db8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "119", - "display": "rotavirus, monovalent" - } - ], - "text": "rotavirus, monovalent" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "date": "2012-06-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4550bd48-907d-4b0c-95a0-a18bffdb0251", - "resource": { - "resourceType": "Immunization", - "id": "4550bd48-907d-4b0c-95a0-a18bffdb0251", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "date": "2012-06-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ab501ee9-b4ac-4264-833e-03fad494bc15", - "resource": { - "resourceType": "Immunization", - "id": "ab501ee9-b4ac-4264-833e-03fad494bc15", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "date": "2012-06-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:cff4121b-3f50-4689-be60-3aeacfc9e0c1", - "resource": { - "resourceType": "Immunization", - "id": "cff4121b-3f50-4689-be60-3aeacfc9e0c1", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - }, - "date": "2012-06-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0baab2f5-8d37-48a1-8819-239b9ee5a693", - "resource": { - "resourceType": "Claim", - "id": "0baab2f5-8d37-48a1-8819-239b9ee5a693", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-06-13T21:05:11-07:00", - "end": "2012-06-13T21:35:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:44758e37-5794-4ca8-b022-1f11b0cbde52" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:21a76df9-25e4-442a-ad18-81a22f061db8" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4550bd48-907d-4b0c-95a0-a18bffdb0251" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:ab501ee9-b4ac-4264-833e-03fad494bc15" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:cff4121b-3f50-4689-be60-3aeacfc9e0c1" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:6d38dd1e-5695-487c-b565-e685839e7149" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2941c94b-b52e-4699-943c-1674670cf890" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 7, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 715.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4d8955ab-386a-4c9a-bffa-6ef71758d8f5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4d8955ab-386a-4c9a-bffa-6ef71758d8f5", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0baab2f5-8d37-48a1-8819-239b9ee5a693" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-06-13T21:35:11-07:00", - "end": "2013-06-13T21:35:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 7, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 143.084, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 572.336, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 715.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 715.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1134.4160000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c", - "resource": { - "resourceType": "Encounter", - "id": "ee5c3a54-49f2-4e2b-910c-648693b9262c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-09-12T21:05:11-07:00", - "end": "2012-09-12T21:20:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4958970a-112e-4948-835d-153f4ced6c47", - "resource": { - "resourceType": "Observation", - "id": "4958970a-112e-4948-835d-153f4ced6c47", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 68.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8d61710f-0619-4525-aa01-53ac24a357d0", - "resource": { - "resourceType": "Observation", - "id": "8d61710f-0619-4525-aa01-53ac24a357d0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a09ec90-77cb-4592-b8c2-18728637f6c7", - "resource": { - "resourceType": "Observation", - "id": "5a09ec90-77cb-4592-b8c2-18728637f6c7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 8.4000, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66d4de90-3817-4805-b2ca-4c8a3c003982", - "resource": { - "resourceType": "Observation", - "id": "66d4de90-3817-4805-b2ca-4c8a3c003982", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 55.623, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e40d929e-2055-422b-9e7b-01298d802388", - "resource": { - "resourceType": "Observation", - "id": "e40d929e-2055-422b-9e7b-01298d802388", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 44.020, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ac68d0c5-92c1-44b9-a234-97d86210e7f7", - "resource": { - "resourceType": "Observation", - "id": "ac68d0c5-92c1-44b9-a234-97d86210e7f7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 130, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c26c785-58df-46a2-bd4b-84ee1c08c5fc", - "resource": { - "resourceType": "Observation", - "id": "7c26c785-58df-46a2-bd4b-84ee1c08c5fc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c48bc12c-275c-437f-a814-ac215a465332", - "resource": { - "resourceType": "Observation", - "id": "c48bc12c-275c-437f-a814-ac215a465332", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7582e76f-3d45-4e06-8cfc-5b02ad752905", - "resource": { - "resourceType": "Observation", - "id": "7582e76f-3d45-4e06-8cfc-5b02ad752905", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "effectiveDateTime": "2012-09-12T21:05:11-07:00", - "issued": "2012-09-12T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a4460d65-8bf3-4a16-84ca-824a41f83bdc", - "resource": { - "resourceType": "Immunization", - "id": "a4460d65-8bf3-4a16-84ca-824a41f83bdc", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "date": "2012-09-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc", - "resource": { - "resourceType": "Immunization", - "id": "71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "date": "2012-09-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7385d405-755e-42e8-bb5b-6ff9796849c8", - "resource": { - "resourceType": "Immunization", - "id": "7385d405-755e-42e8-bb5b-6ff9796849c8", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "date": "2012-09-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:3a9b8b01-3702-4f7e-b547-7ad4c4431e09", - "resource": { - "resourceType": "Immunization", - "id": "3a9b8b01-3702-4f7e-b547-7ad4c4431e09", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "date": "2012-09-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:784285d9-3d56-4283-a96c-a1ce50c9795d", - "resource": { - "resourceType": "Immunization", - "id": "784285d9-3d56-4283-a96c-a1ce50c9795d", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "08", - "display": "Hep B, adolescent or pediatric" - } - ], - "text": "Hep B, adolescent or pediatric" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - }, - "date": "2012-09-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e2a64179-4ad9-47ef-92c2-f46d200f41cd", - "resource": { - "resourceType": "Claim", - "id": "e2a64179-4ad9-47ef-92c2-f46d200f41cd", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-09-12T21:05:11-07:00", - "end": "2012-09-12T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a4460d65-8bf3-4a16-84ca-824a41f83bdc" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:71c99fb4-f1e9-4e7c-abe4-a03929fd4cfc" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7385d405-755e-42e8-bb5b-6ff9796849c8" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:3a9b8b01-3702-4f7e-b547-7ad4c4431e09" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:784285d9-3d56-4283-a96c-a1ce50c9795d" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ee5c3a54-49f2-4e2b-910c-648693b9262c" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:61eabf69-4783-4497-8104-0838dcccbb1e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "61eabf69-4783-4497-8104-0838dcccbb1e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e2a64179-4ad9-47ef-92c2-f46d200f41cd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-09-12T21:20:11-07:00", - "end": "2013-09-12T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd", - "resource": { - "resourceType": "Encounter", - "id": "63b8bb02-ddc1-4dd6-9aba-8143862a16dd", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2012-12-12T20:05:11-08:00", - "end": "2012-12-12T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:13499c66-624f-430a-9dd6-e3dbe3559fb9", - "resource": { - "resourceType": "Observation", - "id": "13499c66-624f-430a-9dd6-e3dbe3559fb9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 72.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4072583b-147b-49dd-99b3-79b677098b91", - "resource": { - "resourceType": "Observation", - "id": "4072583b-147b-49dd-99b3-79b677098b91", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dd90a3df-9750-4d8b-893f-97675b77e789", - "resource": { - "resourceType": "Observation", - "id": "dd90a3df-9750-4d8b-893f-97675b77e789", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 9.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2a2ad20-4ae5-4323-af38-3ee3acc17913", - "resource": { - "resourceType": "Observation", - "id": "b2a2ad20-4ae5-4323-af38-3ee3acc17913", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 62.032, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51ecf38e-63ff-4682-bebe-94dca4ac816f", - "resource": { - "resourceType": "Observation", - "id": "51ecf38e-63ff-4682-bebe-94dca4ac816f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 45.170, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5172691d-9141-4fa4-a095-993cc74512ad", - "resource": { - "resourceType": "Observation", - "id": "5172691d-9141-4fa4-a095-993cc74512ad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bac3973f-718e-4c4f-a0c6-ed166de0eeec", - "resource": { - "resourceType": "Observation", - "id": "bac3973f-718e-4c4f-a0c6-ed166de0eeec", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 68, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29f854bc-0dc7-4f46-a715-ee20acf18402", - "resource": { - "resourceType": "Observation", - "id": "29f854bc-0dc7-4f46-a715-ee20acf18402", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:257317c7-8d18-4847-aba6-04d096e5d9bc", - "resource": { - "resourceType": "Observation", - "id": "257317c7-8d18-4847-aba6-04d096e5d9bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - }, - "effectiveDateTime": "2012-12-12T20:05:11-08:00", - "issued": "2012-12-12T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b7ca4d8-7ebe-4560-b437-fd5eb339b970", - "resource": { - "resourceType": "Claim", - "id": "0b7ca4d8-7ebe-4560-b437-fd5eb339b970", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-12-12T20:05:11-08:00", - "end": "2012-12-12T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:63b8bb02-ddc1-4dd6-9aba-8143862a16dd" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7772f870-a93c-414b-b31e-c57e535a175f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7772f870-a93c-414b-b31e-c57e535a175f", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0b7ca4d8-7ebe-4560-b437-fd5eb339b970" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-12-12T20:20:11-08:00", - "end": "2013-12-12T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", - "resource": { - "resourceType": "Practitioner", - "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10" - } - ], - "active": true, - "name": [ - { - "family": "Jenkins714", - "given": [ - "Lara964" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8", - "resource": { - "resourceType": "Encounter", - "id": "75ca5635-6ff9-42a4-a46c-680aeaa6f8b8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7", - "resource": { - "resourceType": "Condition", - "id": "9e07b9b7-8c28-412f-91d3-b29bde62f6d7", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - }, - "onsetDateTime": "2012-12-29T20:05:11-08:00", - "abatementDateTime": "2013-03-13T21:05:11-07:00", - "assertedDate": "2012-12-29T20:05:11-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:7708603b-7957-4c1e-bfe7-7dd7b25b9290", - "resource": { - "resourceType": "MedicationRequest", - "id": "7708603b-7957-4c1e-bfe7-7dd7b25b9290", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308182", - "display": "Amoxicillin 250 MG Oral Capsule" - } - ], - "text": "Amoxicillin 250 MG Oral Capsule" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - }, - "authoredOn": "2012-12-29T20:05:11-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ec0bf3ae-9dec-420b-9610-229303b44277", - "resource": { - "resourceType": "Claim", - "id": "ec0bf3ae-9dec-420b-9610-229303b44277", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:7708603b-7957-4c1e-bfe7-7dd7b25b9290" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - } - ] - } - ], - "total": { - "value": 3.04, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:938094f3-13b0-451d-b96b-8444272f977b", - "resource": { - "resourceType": "MedicationRequest", - "id": "938094f3-13b0-451d-b96b-8444272f977b", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "198405", - "display": "Ibuprofen 100 MG Oral Tablet" - } - ], - "text": "Ibuprofen 100 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - }, - "authoredOn": "2012-12-29T20:05:11-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:f99eb403-ae1e-4390-8e51-5d4402143221", - "resource": { - "resourceType": "Claim", - "id": "f99eb403-ae1e-4390-8e51-5d4402143221", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:938094f3-13b0-451d-b96b-8444272f977b" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - } - ] - } - ], - "total": { - "value": 40.83, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9", - "resource": { - "resourceType": "Claim", - "id": "1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e4644fb1-bfb4-4ce2-8dbb-561067ea2116", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e4644fb1-bfb4-4ce2-8dbb-561067ea2116", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2012-12-29T20:20:11-08:00", - "end": "2013-12-29T20:20:11-08:00" - }, - "created": "2012-12-29T20:20:11-08:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:1cf13d5e-64bd-4e19-9fd2-b35e9627cfc9" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9e07b9b7-8c28-412f-91d3-b29bde62f6d7" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:75ca5635-6ff9-42a4-a46c-680aeaa6f8b8" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2012-12-29T20:05:11-08:00", - "end": "2012-12-29T20:20:11-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "resource": { - "resourceType": "Organization", - "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAMBRIDGE HEALTH ALLIANCE", - "telecom": [ - { - "system": "phone", - "value": "6176652300" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", - "resource": { - "resourceType": "Practitioner", - "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "40" - } - ], - "active": true, - "name": [ - { - "family": "Gibson10", - "given": [ - "Loretta235" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced", - "resource": { - "resourceType": "Encounter", - "id": "168f43cb-c0ab-4c7d-bd81-5283c6f52ced", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2013-01-31T20:05:11-08:00", - "end": "2013-01-31T20:35:11-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", - "resource": { - "resourceType": "Condition", - "id": "49b1a980-3b3c-48c5-86b8-7c8043daccd9", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" - }, - "onsetDateTime": "2013-01-31T20:05:11-08:00", - "abatementDateTime": "2013-02-11T20:05:11-08:00", - "assertedDate": "2013-01-31T20:05:11-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:f7dd6c01-fa5b-4285-aa86-ae241f4f695f", - "resource": { - "resourceType": "Observation", - "id": "f7dd6c01-fa5b-4285-aa86-ae241f4f695f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" - }, - "effectiveDateTime": "2013-01-31T20:05:11-08:00", - "issued": "2013-01-31T20:05:11.927-08:00", - "valueQuantity": { - "value": 37.100, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9dbe7b30-e60a-4fb3-bed0-751e50b990ae", - "resource": { - "resourceType": "Procedure", - "id": "9dbe7b30-e60a-4fb3-bed0-751e50b990ae", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" - }, - "performedPeriod": { - "start": "2013-01-31T20:05:11-08:00", - "end": "2013-01-31T20:20:11-08:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", - "display": "Acute viral pharyngitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:82c4ce0c-983b-4c32-b6d6-cba1e44ff824", - "resource": { - "resourceType": "Claim", - "id": "82c4ce0c-983b-4c32-b6d6-cba1e44ff824", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-01-31T20:05:11-08:00", - "end": "2013-01-31T20:35:11-08:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:9dbe7b30-e60a-4fb3-bed0-751e50b990ae" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:168f43cb-c0ab-4c7d-bd81-5283c6f52ced" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "net": { - "value": 3121.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f1f8ee6c-acac-40fa-822f-f532927e5a3f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f1f8ee6c-acac-40fa-822f-f532927e5a3f", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "82c4ce0c-983b-4c32-b6d6-cba1e44ff824" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-01-31T20:35:11-08:00", - "end": "2014-01-31T20:35:11-08:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 624.378, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2497.512, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3121.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3121.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2497.512, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053", - "resource": { - "resourceType": "Encounter", - "id": "4f4d4b57-93d6-4990-847d-024cdf717053", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "210098006", - "display": "Domiciliary or rest home patient evaluation and management" - } - ], - "text": "Domiciliary or rest home patient evaluation and management" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2013-02-02T20:05:11-08:00", - "end": "2013-02-02T20:35:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0ab0ed89-1c8f-4fd0-ac1a-17e99365452f", - "resource": { - "resourceType": "Observation", - "id": "0ab0ed89-1c8f-4fd0-ac1a-17e99365452f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "76690-7", - "display": "Sexual orientation" - } - ], - "text": "Sexual orientation" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueString": "heterosexual" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5daf3999-0556-463c-a235-5777e4e45de3", - "resource": { - "resourceType": "Observation", - "id": "5daf3999-0556-463c-a235-5777e4e45de3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "55277-8", - "display": "HIV status" - } - ], - "text": "HIV status" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueString": "not HIV positive" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3a1a780-bcdc-494a-86c2-958549066741", - "resource": { - "resourceType": "Observation", - "id": "e3a1a780-bcdc-494a-86c2-958549066741", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "28245-9", - "display": "Abuse Status [OMAHA]" - } - ], - "text": "Abuse Status [OMAHA]" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueString": "No signs/symptoms" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:417916ff-44af-45a1-a6de-007ebc647401", - "resource": { - "resourceType": "Observation", - "id": "417916ff-44af-45a1-a6de-007ebc647401", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "71802-3", - "display": "Housing status" - } - ], - "text": "Housing status" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueString": "Patient is homeless" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d87effd-d8fa-4f89-bcb5-58a46d580e0e", - "resource": { - "resourceType": "Observation", - "id": "1d87effd-d8fa-4f89-bcb5-58a46d580e0e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "63513-6", - "display": "Are you covered by health insurance or some other kind of health care plan [PhenX]" - } - ], - "text": "Are you covered by health insurance or some other kind of health care plan [PhenX]" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueString": "No" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:78abbc38-83be-4428-a1cf-3a3caefac8f6", - "resource": { - "resourceType": "Observation", - "id": "78abbc38-83be-4428-a1cf-3a3caefac8f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "46240-8", - "display": "History of Hospitalizations+Outpatient visits" - } - ], - "text": "History of Hospitalizations+Outpatient visits" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "effectiveDateTime": "2013-02-02T20:05:11-08:00", - "issued": "2013-02-02T20:05:11.927-08:00", - "valueQuantity": { - "value": 5.7324, - "unit": "{count}", - "system": "http://unitsofmeasure.org", - "code": "{count}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:21081419-0b09-436f-98be-2307d0a81817", - "resource": { - "resourceType": "Procedure", - "id": "21081419-0b09-436f-98be-2307d0a81817", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "311791003", - "display": "Information gathering (procedure)" - } - ], - "text": "Information gathering (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - }, - "performedPeriod": { - "start": "2013-02-02T20:05:11-08:00", - "end": "2013-02-02T20:20:11-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:32027c54-5e42-495a-9c78-8888ebbc35c1", - "resource": { - "resourceType": "Claim", - "id": "32027c54-5e42-495a-9c78-8888ebbc35c1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-02-02T20:05:11-08:00", - "end": "2013-02-02T20:35:11-08:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:21081419-0b09-436f-98be-2307d0a81817" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4f4d4b57-93d6-4990-847d-024cdf717053" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "311791003" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:69381a58-a4f5-4546-a870-724c684559a4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "69381a58-a4f5-4546-a870-724c684559a4", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "32027c54-5e42-495a-9c78-8888ebbc35c1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-02-02T20:35:11-08:00", - "end": "2014-02-02T20:35:11-08:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "311791003" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034", - "resource": { - "resourceType": "Encounter", - "id": "3d4ef3f2-e300-44f8-a47a-e0cc63d47034", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "210098006", - "display": "Domiciliary or rest home patient evaluation and management" - } - ], - "text": "Domiciliary or rest home patient evaluation and management" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2013-03-04T20:05:11-08:00", - "end": "2013-03-04T20:35:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e520cd59-65d9-4e5d-8c4f-2d80ec5953f3", - "resource": { - "resourceType": "Observation", - "id": "e520cd59-65d9-4e5d-8c4f-2d80ec5953f3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "76690-7", - "display": "Sexual orientation" - } - ], - "text": "Sexual orientation" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueString": "heterosexual" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:569ed44c-1187-464c-82a0-da5d39deb5c3", - "resource": { - "resourceType": "Observation", - "id": "569ed44c-1187-464c-82a0-da5d39deb5c3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "55277-8", - "display": "HIV status" - } - ], - "text": "HIV status" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueString": "not HIV positive" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77adb897-9880-4377-953e-c76cd48d7b71", - "resource": { - "resourceType": "Observation", - "id": "77adb897-9880-4377-953e-c76cd48d7b71", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "28245-9", - "display": "Abuse Status [OMAHA]" - } - ], - "text": "Abuse Status [OMAHA]" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueString": "No signs/symptoms" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a7104ede-77ae-4616-9536-f8b23fe2bd10", - "resource": { - "resourceType": "Observation", - "id": "a7104ede-77ae-4616-9536-f8b23fe2bd10", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "71802-3", - "display": "Housing status" - } - ], - "text": "Housing status" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueString": "Patient is homeless" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:89851b54-0acc-4ea9-80f3-e01f9491151b", - "resource": { - "resourceType": "Observation", - "id": "89851b54-0acc-4ea9-80f3-e01f9491151b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "63513-6", - "display": "Are you covered by health insurance or some other kind of health care plan [PhenX]" - } - ], - "text": "Are you covered by health insurance or some other kind of health care plan [PhenX]" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueString": "No" - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ca3e3fa-c66b-4357-a0b9-1c3171d3f8ad", - "resource": { - "resourceType": "Observation", - "id": "9ca3e3fa-c66b-4357-a0b9-1c3171d3f8ad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "social-history", - "display": "social-history" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "46240-8", - "display": "History of Hospitalizations+Outpatient visits" - } - ], - "text": "History of Hospitalizations+Outpatient visits" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "effectiveDateTime": "2013-03-04T20:05:11-08:00", - "issued": "2013-03-04T20:05:11.927-08:00", - "valueQuantity": { - "value": 8.6061, - "unit": "{count}", - "system": "http://unitsofmeasure.org", - "code": "{count}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6", - "resource": { - "resourceType": "Procedure", - "id": "6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "311791003", - "display": "Information gathering (procedure)" - } - ], - "text": "Information gathering (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - }, - "performedPeriod": { - "start": "2013-03-04T20:05:11-08:00", - "end": "2013-03-04T20:20:11-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4303b920-9215-4025-b38f-44220da09e24", - "resource": { - "resourceType": "Claim", - "id": "4303b920-9215-4025-b38f-44220da09e24", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-03-04T20:05:11-08:00", - "end": "2013-03-04T20:35:11-08:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:6fe519ea-c1b3-49ea-bdfa-2de10c77a4f6" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3d4ef3f2-e300-44f8-a47a-e0cc63d47034" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "311791003" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:28831e19-70c7-41e4-affa-58be1b22c66a", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "28831e19-70c7-41e4-affa-58be1b22c66a", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4303b920-9215-4025-b38f-44220da09e24" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-03-04T20:35:11-08:00", - "end": "2014-03-04T20:35:11-08:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "311791003" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443", - "resource": { - "resourceType": "Encounter", - "id": "1ac04561-97d1-4983-b0bc-3464237c7443", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2013-03-13T21:05:11-07:00", - "end": "2013-03-13T21:20:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:07bfee88-8db5-4fa7-90a6-01638ee9e6d1", - "resource": { - "resourceType": "Observation", - "id": "07bfee88-8db5-4fa7-90a6-01638ee9e6d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 76, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12943f61-67a0-4425-be25-9d02b7243f2f", - "resource": { - "resourceType": "Observation", - "id": "12943f61-67a0-4425-be25-9d02b7243f2f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0156d6ff-3378-4971-a882-8b8bb2aa5348", - "resource": { - "resourceType": "Observation", - "id": "0156d6ff-3378-4971-a882-8b8bb2aa5348", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 10.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:434e13bd-8671-48bb-bb9a-53a9d6ff5781", - "resource": { - "resourceType": "Observation", - "id": "434e13bd-8671-48bb-bb9a-53a9d6ff5781", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 61.221, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8ad5932-ae64-458c-8554-b0dd95fd11a6", - "resource": { - "resourceType": "Observation", - "id": "d8ad5932-ae64-458c-8554-b0dd95fd11a6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 45.990, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5eda79fe-29aa-494b-a5d0-1178a1e419d1", - "resource": { - "resourceType": "Observation", - "id": "5eda79fe-29aa-494b-a5d0-1178a1e419d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 88, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f4e2d59d-0c48-47d7-8ea2-aa5812897c69", - "resource": { - "resourceType": "Observation", - "id": "f4e2d59d-0c48-47d7-8ea2-aa5812897c69", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 73, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7b8c59c9-eb6b-4552-808e-033e23182ded", - "resource": { - "resourceType": "Observation", - "id": "7b8c59c9-eb6b-4552-808e-033e23182ded", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:23039a4c-fb0d-4d95-9ebd-6050f6fee708", - "resource": { - "resourceType": "Observation", - "id": "23039a4c-fb0d-4d95-9ebd-6050f6fee708", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "effectiveDateTime": "2013-03-13T21:05:11-07:00", - "issued": "2013-03-13T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cb2ea76-d545-431a-a26b-08691dcf6ab1", - "resource": { - "resourceType": "Immunization", - "id": "0cb2ea76-d545-431a-a26b-08691dcf6ab1", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "49", - "display": "Hib (PRP-OMP)" - } - ], - "text": "Hib (PRP-OMP)" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "date": "2013-03-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c", - "resource": { - "resourceType": "Immunization", - "id": "a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "date": "2013-03-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c", - "resource": { - "resourceType": "Immunization", - "id": "5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "date": "2013-03-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e581b310-2155-4126-89b9-8bc9a79483df", - "resource": { - "resourceType": "Immunization", - "id": "e581b310-2155-4126-89b9-8bc9a79483df", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "date": "2013-03-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:49f5a02d-bda1-4fd6-a46c-8e438a131ac0", - "resource": { - "resourceType": "Immunization", - "id": "49f5a02d-bda1-4fd6-a46c-8e438a131ac0", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - }, - "date": "2013-03-13T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14", - "resource": { - "resourceType": "Claim", - "id": "1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-03-13T21:05:11-07:00", - "end": "2013-03-13T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:0cb2ea76-d545-431a-a26b-08691dcf6ab1" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a5dbff83-1b5e-477c-b4d2-f3cf30b0a59c" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:5c8edf6e-c6a1-42f4-9d8f-f59a8162cc4c" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e581b310-2155-4126-89b9-8bc9a79483df" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:49f5a02d-bda1-4fd6-a46c-8e438a131ac0" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1ac04561-97d1-4983-b0bc-3464237c7443" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:739688b2-d627-48d0-aad8-d7bb600b407d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "739688b2-d627-48d0-aad8-d7bb600b407d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1b40afd5-c6db-4ba7-9a6b-0eaec99d2c14" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-03-13T21:20:11-07:00", - "end": "2014-03-13T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629", - "resource": { - "resourceType": "Encounter", - "id": "04a40a21-1488-4fab-a0f0-832765c91629", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4", - "resource": { - "resourceType": "Condition", - "id": "cf4fa5f3-360e-4adf-8dbe-48724beac3b4", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - }, - "onsetDateTime": "2013-05-12T21:05:11-07:00", - "abatementDateTime": "2013-06-12T21:05:11-07:00", - "assertedDate": "2013-05-12T21:05:11-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ff4c2f70-0740-44d9-bdcf-9a413399b902", - "resource": { - "resourceType": "MedicationRequest", - "id": "ff4c2f70-0740-44d9-bdcf-9a413399b902", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308182", - "display": "Amoxicillin 250 MG Oral Capsule" - } - ], - "text": "Amoxicillin 250 MG Oral Capsule" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - }, - "authoredOn": "2013-05-12T21:05:11-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:768f2ce4-4906-43ec-b75e-4539acb28c38", - "resource": { - "resourceType": "Claim", - "id": "768f2ce4-4906-43ec-b75e-4539acb28c38", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:ff4c2f70-0740-44d9-bdcf-9a413399b902" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - } - ] - } - ], - "total": { - "value": 4.11, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a7858988-a2d3-425d-871e-50e8ba7fd234", - "resource": { - "resourceType": "MedicationRequest", - "id": "a7858988-a2d3-425d-871e-50e8ba7fd234", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - }, - "authoredOn": "2013-05-12T21:05:11-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:8f93911c-303b-4c1a-8660-3a4ceaa7c16e", - "resource": { - "resourceType": "Claim", - "id": "8f93911c-303b-4c1a-8660-3a4ceaa7c16e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:a7858988-a2d3-425d-871e-50e8ba7fd234" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - } - ] - } - ], - "total": { - "value": 3.24, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ba298548-2f1c-4050-8f2d-01f8a0dce4b2", - "resource": { - "resourceType": "Claim", - "id": "ba298548-2f1c-4050-8f2d-01f8a0dce4b2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5fc4068c-ceb6-4b2c-9923-e6c54e7c22b5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5fc4068c-ceb6-4b2c-9923-e6c54e7c22b5", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ba298548-2f1c-4050-8f2d-01f8a0dce4b2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-05-12T21:20:11-07:00", - "end": "2014-05-12T21:20:11-07:00" - }, - "created": "2013-05-12T21:20:11-07:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:ba298548-2f1c-4050-8f2d-01f8a0dce4b2" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:cf4fa5f3-360e-4adf-8dbe-48724beac3b4" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:04a40a21-1488-4fab-a0f0-832765c91629" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2013-05-12T21:05:11-07:00", - "end": "2013-05-12T21:20:11-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d", - "resource": { - "resourceType": "Encounter", - "id": "630b6bc9-4bce-421d-b42b-ced0b61bfc1d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2013-06-12T21:05:11-07:00", - "end": "2013-06-12T21:20:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:16c1366e-020a-4683-b698-8aeb6b4f9de9", - "resource": { - "resourceType": "Observation", - "id": "16c1366e-020a-4683-b698-8aeb6b4f9de9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 78.900, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:43c11759-e774-4878-a48d-c2e6b8bc171d", - "resource": { - "resourceType": "Observation", - "id": "43c11759-e774-4878-a48d-c2e6b8bc171d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:238cfd3f-a55e-4221-bddf-6375dc0ae15c", - "resource": { - "resourceType": "Observation", - "id": "238cfd3f-a55e-4221-bddf-6375dc0ae15c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 10.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:93bde22e-3692-4bef-8ee4-1d603ab44e2d", - "resource": { - "resourceType": "Observation", - "id": "93bde22e-3692-4bef-8ee4-1d603ab44e2d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 56.580, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eb3c872c-da8a-4dca-a2f1-48c56ea136ef", - "resource": { - "resourceType": "Observation", - "id": "eb3c872c-da8a-4dca-a2f1-48c56ea136ef", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 46.610, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:87cf9b73-8553-4570-be0d-92e3a07c3664", - "resource": { - "resourceType": "Observation", - "id": "87cf9b73-8553-4570-be0d-92e3a07c3664", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bb09f1d-f929-44c2-b883-366f77580380", - "resource": { - "resourceType": "Observation", - "id": "7bb09f1d-f929-44c2-b883-366f77580380", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 71, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b4f036b-9727-4701-816e-f0c9c8971102", - "resource": { - "resourceType": "Observation", - "id": "8b4f036b-9727-4701-816e-f0c9c8971102", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7dc653f0-d600-486a-8f6d-28e375dceb92", - "resource": { - "resourceType": "Observation", - "id": "7dc653f0-d600-486a-8f6d-28e375dceb92", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "effectiveDateTime": "2013-06-12T21:05:11-07:00", - "issued": "2013-06-12T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d87f5df-cbc2-4520-b461-ae3f947f84b3", - "resource": { - "resourceType": "Immunization", - "id": "6d87f5df-cbc2-4520-b461-ae3f947f84b3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - }, - "date": "2013-06-12T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:0d0d8ca4-0f73-4770-8f20-fa13d3bafa31", - "resource": { - "resourceType": "Claim", - "id": "0d0d8ca4-0f73-4770-8f20-fa13d3bafa31", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-06-12T21:05:11-07:00", - "end": "2013-06-12T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:6d87f5df-cbc2-4520-b461-ae3f947f84b3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:630b6bc9-4bce-421d-b42b-ced0b61bfc1d" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a4960828-5830-4f8f-94a8-950cb4c3278c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a4960828-5830-4f8f-94a8-950cb4c3278c", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0d0d8ca4-0f73-4770-8f20-fa13d3bafa31" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-06-12T21:20:11-07:00", - "end": "2014-06-12T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220", - "resource": { - "resourceType": "Encounter", - "id": "e6e37c5f-169c-4541-9398-654855632220", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2013-12-11T20:05:11-08:00", - "end": "2013-12-11T20:35:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:710bb7ee-7de5-4d47-baf1-49fb0111def1", - "resource": { - "resourceType": "Observation", - "id": "710bb7ee-7de5-4d47-baf1-49fb0111def1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 84, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c9397236-bb65-4d44-a78e-ec4543b323e6", - "resource": { - "resourceType": "Observation", - "id": "c9397236-bb65-4d44-a78e-ec4543b323e6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ca6345d4-576b-413d-b695-245e95aebf7d", - "resource": { - "resourceType": "Observation", - "id": "ca6345d4-576b-413d-b695-245e95aebf7d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 11.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36d1f125-237a-4fbc-becf-116c878ee167", - "resource": { - "resourceType": "Observation", - "id": "36d1f125-237a-4fbc-becf-116c878ee167", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 50.402, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f5b04bb3-8fbe-4f97-99b2-afdb04369b7e", - "resource": { - "resourceType": "Observation", - "id": "f5b04bb3-8fbe-4f97-99b2-afdb04369b7e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 47.460, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6eb67837-fc64-436b-ad5f-30ff74eca795", - "resource": { - "resourceType": "Observation", - "id": "6eb67837-fc64-436b-ad5f-30ff74eca795", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df217223-0909-45d0-b6ef-8f211a25187d", - "resource": { - "resourceType": "Observation", - "id": "df217223-0909-45d0-b6ef-8f211a25187d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 89, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ffd13286-968f-48dd-ba24-6d8cfb2cfdb0", - "resource": { - "resourceType": "Observation", - "id": "ffd13286-968f-48dd-ba24-6d8cfb2cfdb0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:241cfb35-9d6f-4654-b76d-23256bdd6990", - "resource": { - "resourceType": "Observation", - "id": "241cfb35-9d6f-4654-b76d-23256bdd6990", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "effectiveDateTime": "2013-12-11T20:05:11-08:00", - "issued": "2013-12-11T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2c9bca50-2512-4511-b04f-a85aaa536f19", - "resource": { - "resourceType": "Procedure", - "id": "2c9bca50-2512-4511-b04f-a85aaa536f19", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "performedPeriod": { - "start": "2013-12-11T20:05:11-08:00", - "end": "2013-12-11T20:20:11-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7f811192-0c03-4448-a371-f349b14e0e0d", - "resource": { - "resourceType": "Immunization", - "id": "7f811192-0c03-4448-a371-f349b14e0e0d", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - }, - "date": "2013-12-11T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1c023d1e-1c27-47aa-8af8-4e40a119f9f6", - "resource": { - "resourceType": "Claim", - "id": "1c023d1e-1c27-47aa-8af8-4e40a119f9f6", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2013-12-11T20:05:11-08:00", - "end": "2013-12-11T20:35:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7f811192-0c03-4448-a371-f349b14e0e0d" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:2c9bca50-2512-4511-b04f-a85aaa536f19" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e6e37c5f-169c-4541-9398-654855632220" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 788.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:15f60b34-ddf7-41ef-affa-73a0b93e40cf", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "15f60b34-ddf7-41ef-affa-73a0b93e40cf", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1c023d1e-1c27-47aa-8af8-4e40a119f9f6" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-12-11T20:35:11-08:00", - "end": "2014-12-11T20:35:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 157.686, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 630.744, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 788.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 788.43, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 743.1600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba", - "resource": { - "resourceType": "Encounter", - "id": "3176da2e-1231-4ba8-95e6-41e5c9de93ba", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2014-06-11T21:05:11-07:00", - "end": "2014-06-11T21:35:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9137e6df-7c09-477b-87ef-e017d1f4f45e", - "resource": { - "resourceType": "Observation", - "id": "9137e6df-7c09-477b-87ef-e017d1f4f45e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 87.700, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0aeab6bc-740e-4e22-ad25-450dd5e3eb9c", - "resource": { - "resourceType": "Observation", - "id": "0aeab6bc-740e-4e22-ad25-450dd5e3eb9c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:34c35f67-19be-4c3d-958e-d5ece010b4c6", - "resource": { - "resourceType": "Observation", - "id": "34c35f67-19be-4c3d-958e-d5ece010b4c6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 12.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1bcfb693-e2ab-4ea7-8687-b3953fa078e2", - "resource": { - "resourceType": "Observation", - "id": "1bcfb693-e2ab-4ea7-8687-b3953fa078e2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 37.678, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f3bdfcc7-9d23-46e8-9d6a-f1a6c867666d", - "resource": { - "resourceType": "Observation", - "id": "f3bdfcc7-9d23-46e8-9d6a-f1a6c867666d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 47.990, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d52542af-7a4a-4a1c-a332-6052018ed37e", - "resource": { - "resourceType": "Observation", - "id": "d52542af-7a4a-4a1c-a332-6052018ed37e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 16.160, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c300b10-ecff-4296-a8c0-4bd15efbf4d1", - "resource": { - "resourceType": "Observation", - "id": "0c300b10-ecff-4296-a8c0-4bd15efbf4d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 45.562, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cca9da08-699c-4152-89a4-7cae0ed7a1fe", - "resource": { - "resourceType": "Observation", - "id": "cca9da08-699c-4152-89a4-7cae0ed7a1fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 104, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9dc6d8c7-c78d-4202-ab0a-99d32f1795c8", - "resource": { - "resourceType": "Observation", - "id": "9dc6d8c7-c78d-4202-ab0a-99d32f1795c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 62, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:58a8ff38-e132-49ed-88aa-f670ddc47063", - "resource": { - "resourceType": "Observation", - "id": "58a8ff38-e132-49ed-88aa-f670ddc47063", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:acad2f45-f255-4cc1-a376-f3ceb0ce414a", - "resource": { - "resourceType": "Observation", - "id": "acad2f45-f255-4cc1-a376-f3ceb0ce414a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "effectiveDateTime": "2014-06-11T21:05:11-07:00", - "issued": "2014-06-11T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:51eae485-2625-40aa-8b20-9e1d0e1e61f9", - "resource": { - "resourceType": "Procedure", - "id": "51eae485-2625-40aa-8b20-9e1d0e1e61f9", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "performedPeriod": { - "start": "2014-06-11T21:05:11-07:00", - "end": "2014-06-11T21:20:11-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c3e287d7-8bb8-4c97-b719-0cb0fc3f650f", - "resource": { - "resourceType": "Immunization", - "id": "c3e287d7-8bb8-4c97-b719-0cb0fc3f650f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - }, - "date": "2014-06-11T21:05:11-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e", - "resource": { - "resourceType": "Claim", - "id": "113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-06-11T21:05:11-07:00", - "end": "2014-06-11T21:35:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c3e287d7-8bb8-4c97-b719-0cb0fc3f650f" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:51eae485-2625-40aa-8b20-9e1d0e1e61f9" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3176da2e-1231-4ba8-95e6-41e5c9de93ba" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 898.40, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:967b251b-ee1c-491d-ac21-3d21df8df8a9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "967b251b-ee1c-491d-ac21-3d21df8df8a9", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "113bd33a-39ed-4d9a-a3e8-8dfa2d9d0f7e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-06-11T21:35:11-07:00", - "end": "2015-06-11T21:35:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 179.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 718.72, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 898.40, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 898.40, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 831.1360000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357", - "resource": { - "resourceType": "Encounter", - "id": "f07a921a-582d-437e-b19e-46d8d6da8357", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0", - "resource": { - "resourceType": "Condition", - "id": "74e3b413-1939-43e9-b43d-782a1f5b13f0", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - }, - "onsetDateTime": "2014-11-04T20:05:11-08:00", - "abatementDateTime": "2014-12-10T20:05:11-08:00", - "assertedDate": "2014-11-04T20:05:11-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ea6e0ecc-fbbd-4bad-ba83-4875393c71fd", - "resource": { - "resourceType": "MedicationRequest", - "id": "ea6e0ecc-fbbd-4bad-ba83-4875393c71fd", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308182", - "display": "Amoxicillin 250 MG Oral Capsule" - } - ], - "text": "Amoxicillin 250 MG Oral Capsule" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - }, - "authoredOn": "2014-11-04T20:05:11-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:bddac4ad-576b-4027-9324-75f21ffcba27", - "resource": { - "resourceType": "Claim", - "id": "bddac4ad-576b-4027-9324-75f21ffcba27", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:ea6e0ecc-fbbd-4bad-ba83-4875393c71fd" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - } - ] - } - ], - "total": { - "value": 8.05, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c35e09f3-b451-4765-b703-d2dc13d7b340", - "resource": { - "resourceType": "MedicationRequest", - "id": "c35e09f3-b451-4765-b703-d2dc13d7b340", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "198405", - "display": "Ibuprofen 100 MG Oral Tablet" - } - ], - "text": "Ibuprofen 100 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - }, - "authoredOn": "2014-11-04T20:05:11-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1e02d38a-86b1-4ce9-a848-fba736c1ec7d", - "resource": { - "resourceType": "Claim", - "id": "1e02d38a-86b1-4ce9-a848-fba736c1ec7d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:c35e09f3-b451-4765-b703-d2dc13d7b340" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - } - ] - } - ], - "total": { - "value": 37.1, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:91e1a143-5672-4954-a626-ddeb4a0b9974", - "resource": { - "resourceType": "Claim", - "id": "91e1a143-5672-4954-a626-ddeb4a0b9974", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:99de1846-3de5-4fc7-91ca-c6770cc86bee", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "99de1846-3de5-4fc7-91ca-c6770cc86bee", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "91e1a143-5672-4954-a626-ddeb4a0b9974" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-11-04T20:20:11-08:00", - "end": "2015-11-04T20:20:11-08:00" - }, - "created": "2014-11-04T20:20:11-08:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:91e1a143-5672-4954-a626-ddeb4a0b9974" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:74e3b413-1939-43e9-b43d-782a1f5b13f0" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f07a921a-582d-437e-b19e-46d8d6da8357" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-11-04T20:05:11-08:00", - "end": "2014-11-04T20:20:11-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d", - "resource": { - "resourceType": "Encounter", - "id": "92c83229-bcec-4d68-bd71-d282d7f0052d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2014-12-10T20:05:11-08:00", - "end": "2014-12-10T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:56655e3a-4be3-4121-a5f9-a1b1ae7ee983", - "resource": { - "resourceType": "Observation", - "id": "56655e3a-4be3-4121-a5f9-a1b1ae7ee983", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 91.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:26ee22da-ae0a-4e47-a934-caa664265146", - "resource": { - "resourceType": "Observation", - "id": "26ee22da-ae0a-4e47-a934-caa664265146", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77c6092e-197f-4338-a216-bdea9f6598a0", - "resource": { - "resourceType": "Observation", - "id": "77c6092e-197f-4338-a216-bdea9f6598a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 12.700, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d72f431b-d180-4098-83ee-ac98bd314455", - "resource": { - "resourceType": "Observation", - "id": "d72f431b-d180-4098-83ee-ac98bd314455", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 17.050, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d08a9cfd-81d7-40b7-8565-2af0ce9ba012", - "resource": { - "resourceType": "Observation", - "id": "d08a9cfd-81d7-40b7-8565-2af0ce9ba012", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 48.320, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9fa7636a-dcca-4748-9421-c9a2e8fffaa6", - "resource": { - "resourceType": "Observation", - "id": "9fa7636a-dcca-4748-9421-c9a2e8fffaa6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 15.170, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9dc13a84-663c-49db-b494-b656d9d250a2", - "resource": { - "resourceType": "Observation", - "id": "9dc13a84-663c-49db-b494-b656d9d250a2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 21.242, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4e78f834-8a3c-4541-bb4c-b582e575f620", - "resource": { - "resourceType": "Observation", - "id": "4e78f834-8a3c-4541-bb4c-b582e575f620", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 78, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9f9ae521-9b30-4791-be08-698827a0d63d", - "resource": { - "resourceType": "Observation", - "id": "9f9ae521-9b30-4791-be08-698827a0d63d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 61, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:98c2b2c4-5917-42ab-8338-91dde415d58d", - "resource": { - "resourceType": "Observation", - "id": "98c2b2c4-5917-42ab-8338-91dde415d58d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2967b643-54cb-4608-806d-76c019076056", - "resource": { - "resourceType": "Observation", - "id": "2967b643-54cb-4608-806d-76c019076056", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "effectiveDateTime": "2014-12-10T20:05:11-08:00", - "issued": "2014-12-10T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50ed6852-2852-42a7-9ec8-c187270a8d1f", - "resource": { - "resourceType": "Immunization", - "id": "50ed6852-2852-42a7-9ec8-c187270a8d1f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - }, - "date": "2014-12-10T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2", - "resource": { - "resourceType": "Claim", - "id": "ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2014-12-10T20:05:11-08:00", - "end": "2014-12-10T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:50ed6852-2852-42a7-9ec8-c187270a8d1f" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:92c83229-bcec-4d68-bd71-d282d7f0052d" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8f1e1663-b4ce-46e8-aaf1-0b31cca0a9a0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8f1e1663-b4ce-46e8-aaf1-0b31cca0a9a0", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ce28fe69-df7d-4aee-a19e-5f7ed6a8fcf2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-12-10T20:20:11-08:00", - "end": "2015-12-10T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e", - "resource": { - "resourceType": "Encounter", - "id": "3a063f0c-80ce-479e-a6f1-01a6421a9f0e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2015-05-06T21:05:11-07:00", - "end": "2015-05-06T21:35:11-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e", - "resource": { - "resourceType": "Condition", - "id": "0ee4180b-3fcb-4850-a816-f745e3a6b79e", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" - }, - "onsetDateTime": "2015-05-06T21:05:11-07:00", - "abatementDateTime": "2015-05-15T21:05:11-07:00", - "assertedDate": "2015-05-06T21:05:11-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ecf6e70e-bdfe-4421-a48d-e257a5514ad6", - "resource": { - "resourceType": "Observation", - "id": "ecf6e70e-bdfe-4421-a48d-e257a5514ad6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" - }, - "effectiveDateTime": "2015-05-06T21:05:11-07:00", - "issued": "2015-05-06T21:05:11.927-07:00", - "valueQuantity": { - "value": 37.538, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4a1db053-bc05-452b-8f91-0b1ea35a94eb", - "resource": { - "resourceType": "Procedure", - "id": "4a1db053-bc05-452b-8f91-0b1ea35a94eb", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" - }, - "performedPeriod": { - "start": "2015-05-06T21:05:11-07:00", - "end": "2015-05-06T21:20:11-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:49b1a980-3b3c-48c5-86b8-7c8043daccd9", - "display": "Acute viral pharyngitis (disorder)" - }, - { - "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e", - "display": "Acute viral pharyngitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:ae5b845e-062b-40d9-a4ee-65db59760c8d", - "resource": { - "resourceType": "Claim", - "id": "ae5b845e-062b-40d9-a4ee-65db59760c8d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2015-05-06T21:05:11-07:00", - "end": "2015-05-06T21:35:11-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4a1db053-bc05-452b-8f91-0b1ea35a94eb" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3a063f0c-80ce-479e-a6f1-01a6421a9f0e" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "net": { - "value": 2578.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c2ba8d5d-514f-45ef-92fb-7ebdb37de26b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c2ba8d5d-514f-45ef-92fb-7ebdb37de26b", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ae5b845e-062b-40d9-a4ee-65db59760c8d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-05-06T21:35:11-07:00", - "end": "2016-05-06T21:35:11-07:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:0ee4180b-3fcb-4850-a816-f745e3a6b79e" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 515.616, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2062.464, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2578.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2578.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2062.464, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe", - "resource": { - "resourceType": "Encounter", - "id": "a1d5506d-6503-44e6-8aa0-73c86fb538fe", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2015-06-10T21:05:11-07:00", - "end": "2015-06-10T21:35:11-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:20e18781-ca54-41a6-be68-fee4f84bc20e", - "resource": { - "resourceType": "Observation", - "id": "20e18781-ca54-41a6-be68-fee4f84bc20e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 95.200, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d217e7a9-30af-4bc0-be0a-0829b53c748d", - "resource": { - "resourceType": "Observation", - "id": "d217e7a9-30af-4bc0-be0a-0829b53c748d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9397dd21-32d9-44bb-88f4-4f738ead8ca3", - "resource": { - "resourceType": "Observation", - "id": "9397dd21-32d9-44bb-88f4-4f738ead8ca3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 13.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:02bf437b-44b2-4b93-b3f3-985fdea74f33", - "resource": { - "resourceType": "Observation", - "id": "02bf437b-44b2-4b93-b3f3-985fdea74f33", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 19.973, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df1d7890-006b-4dc0-a043-cf88ef538f09", - "resource": { - "resourceType": "Observation", - "id": "df1d7890-006b-4dc0-a043-cf88ef538f09", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 48.340, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4586c508-3e96-45e5-bd88-410e97ee96ea", - "resource": { - "resourceType": "Observation", - "id": "4586c508-3e96-45e5-bd88-410e97ee96ea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 14.520, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e10536a-5aec-4765-9807-e61ec16e5b60", - "resource": { - "resourceType": "Observation", - "id": "8e10536a-5aec-4765-9807-e61ec16e5b60", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 10.006, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a238fa6-4fae-4381-81e4-820f9cca4cb9", - "resource": { - "resourceType": "Observation", - "id": "8a238fa6-4fae-4381-81e4-820f9cca4cb9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 124, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:26601664-5b83-4000-a4d3-d5079556dc57", - "resource": { - "resourceType": "Observation", - "id": "26601664-5b83-4000-a4d3-d5079556dc57", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 62, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1c82fc69-c853-4366-ae7a-a1008fc02265", - "resource": { - "resourceType": "Observation", - "id": "1c82fc69-c853-4366-ae7a-a1008fc02265", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e2fb7236-49fa-42cc-9833-9d16aab4d918", - "resource": { - "resourceType": "Observation", - "id": "e2fb7236-49fa-42cc-9833-9d16aab4d918", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "effectiveDateTime": "2015-06-10T21:05:11-07:00", - "issued": "2015-06-10T21:05:11.927-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cf24ed92-3dc9-4bc0-b850-b252f0e7d200", - "resource": { - "resourceType": "Procedure", - "id": "cf24ed92-3dc9-4bc0-b850-b252f0e7d200", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - }, - "performedPeriod": { - "start": "2015-06-10T21:05:11-07:00", - "end": "2015-06-10T21:20:11-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:45983547-9f85-44d3-aaaf-60897a1fe192", - "resource": { - "resourceType": "Claim", - "id": "45983547-9f85-44d3-aaaf-60897a1fe192", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2015-06-10T21:05:11-07:00", - "end": "2015-06-10T21:35:11-07:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:cf24ed92-3dc9-4bc0-b850-b252f0e7d200" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:a1d5506d-6503-44e6-8aa0-73c86fb538fe" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 874.29, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d7deff37-292b-46b6-aa2f-544798e0966e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d7deff37-292b-46b6-aa2f-544798e0966e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "45983547-9f85-44d3-aaaf-60897a1fe192" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-06-10T21:35:11-07:00", - "end": "2016-06-10T21:35:11-07:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 174.858, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 699.432, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 874.29, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 874.29, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 699.432, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd", - "resource": { - "resourceType": "Encounter", - "id": "636f051c-bece-4ff2-b60a-a896bd3c11cd", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2015-09-20T21:05:11-07:00", - "end": "2015-09-20T21:20:11-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6", - "resource": { - "resourceType": "Condition", - "id": "03a85dd0-bd02-4318-ac7f-33d0b30903b6", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd" - }, - "onsetDateTime": "2015-09-20T21:05:11-07:00", - "abatementDateTime": "2015-10-04T21:05:11-07:00", - "assertedDate": "2015-09-20T21:05:11-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:e45a1099-63af-4d04-b1dd-fc0a4e6bbab6", - "resource": { - "resourceType": "Claim", - "id": "e45a1099-63af-4d04-b1dd-fc0a4e6bbab6", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2015-09-20T21:05:11-07:00", - "end": "2015-09-20T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:636f051c-bece-4ff2-b60a-a896bd3c11cd" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7322204f-9eb4-4c12-9082-8ec045f053cd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7322204f-9eb4-4c12-9082-8ec045f053cd", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e45a1099-63af-4d04-b1dd-fc0a4e6bbab6" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-09-20T21:20:11-07:00", - "end": "2016-09-20T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:03a85dd0-bd02-4318-ac7f-33d0b30903b6" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629", - "resource": { - "resourceType": "Encounter", - "id": "67c12bfb-95a8-42c6-9340-a15cc1a55629", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2015-12-09T20:05:11-08:00", - "end": "2015-12-09T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:520c948e-4089-4883-a104-f5764e8964f2", - "resource": { - "resourceType": "Observation", - "id": "520c948e-4089-4883-a104-f5764e8964f2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 98.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36eb9e11-5780-43ab-b1c7-b0dcaf821ca4", - "resource": { - "resourceType": "Observation", - "id": "36eb9e11-5780-43ab-b1c7-b0dcaf821ca4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:64a8ed52-5e90-4906-81c8-cca235a848c6", - "resource": { - "resourceType": "Observation", - "id": "64a8ed52-5e90-4906-81c8-cca235a848c6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 14.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b70bd543-daad-4fdc-9b6b-6528add2fc8c", - "resource": { - "resourceType": "Observation", - "id": "b70bd543-daad-4fdc-9b6b-6528add2fc8c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 19.973, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6078066b-daea-4b15-a319-eea35bd76573", - "resource": { - "resourceType": "Observation", - "id": "6078066b-daea-4b15-a319-eea35bd76573", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 48.340, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dbf72292-37e0-4c94-b475-6d1c9c1af60f", - "resource": { - "resourceType": "Observation", - "id": "dbf72292-37e0-4c94-b475-6d1c9c1af60f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 14.730, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7ada8175-86e1-471a-a361-f6a23bb332cf", - "resource": { - "resourceType": "Observation", - "id": "7ada8175-86e1-471a-a361-f6a23bb332cf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 18.646, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6cc1395e-0f96-4ca5-bcf8-a71957ed8f24", - "resource": { - "resourceType": "Observation", - "id": "6cc1395e-0f96-4ca5-bcf8-a71957ed8f24", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 117, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:44bef88f-f1f9-4702-b063-54ef033a4307", - "resource": { - "resourceType": "Observation", - "id": "44bef88f-f1f9-4702-b063-54ef033a4307", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 76, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8b348046-2f08-446d-80c0-e982b83637bc", - "resource": { - "resourceType": "Observation", - "id": "8b348046-2f08-446d-80c0-e982b83637bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:539a3c6d-5f18-4225-a839-a47f8a0e0c68", - "resource": { - "resourceType": "Observation", - "id": "539a3c6d-5f18-4225-a839-a47f8a0e0c68", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "effectiveDateTime": "2015-12-09T20:05:11-08:00", - "issued": "2015-12-09T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4259fe9b-5bfe-4be4-a8b7-707e44be6dde", - "resource": { - "resourceType": "Immunization", - "id": "4259fe9b-5bfe-4be4-a8b7-707e44be6dde", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - }, - "date": "2015-12-09T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d640d081-ad7e-408b-b692-6c32c7c53864", - "resource": { - "resourceType": "Claim", - "id": "d640d081-ad7e-408b-b692-6c32c7c53864", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2015-12-09T20:05:11-08:00", - "end": "2015-12-09T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4259fe9b-5bfe-4be4-a8b7-707e44be6dde" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:67c12bfb-95a8-42c6-9340-a15cc1a55629" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3275cdec-5461-4170-be46-6efc703d0e5c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3275cdec-5461-4170-be46-6efc703d0e5c", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d640d081-ad7e-408b-b692-6c32c7c53864" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-12-09T20:20:11-08:00", - "end": "2016-12-09T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810", - "resource": { - "resourceType": "Encounter", - "id": "3bd80262-f307-4c18-8198-72d23d3bf810", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - } - ], - "period": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65", - "resource": { - "resourceType": "Condition", - "id": "aa9df8bf-c94c-40a2-8483-fcfab7698c65", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - }, - "onsetDateTime": "2016-08-01T21:05:11-07:00", - "abatementDateTime": "2016-12-14T20:05:11-08:00", - "assertedDate": "2016-08-01T21:05:11-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:326cc4c9-c367-467b-9698-1279efd7f02a", - "resource": { - "resourceType": "MedicationRequest", - "id": "326cc4c9-c367-467b-9698-1279efd7f02a", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308182", - "display": "Amoxicillin 250 MG Oral Capsule" - } - ], - "text": "Amoxicillin 250 MG Oral Capsule" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - }, - "authoredOn": "2016-08-01T21:05:11-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:7e1318f4-6d5e-4ac3-a33f-6435543c9833", - "resource": { - "resourceType": "Claim", - "id": "7e1318f4-6d5e-4ac3-a33f-6435543c9833", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:326cc4c9-c367-467b-9698-1279efd7f02a" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - } - ] - } - ], - "total": { - "value": 4.61, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a9493add-b2e9-46f9-bc58-6502eb98b460", - "resource": { - "resourceType": "MedicationRequest", - "id": "a9493add-b2e9-46f9-bc58-6502eb98b460", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "198405", - "display": "Ibuprofen 100 MG Oral Tablet" - } - ], - "text": "Ibuprofen 100 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - }, - "authoredOn": "2016-08-01T21:05:11-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840" - }, - "onBehalfOf": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:4d3ba025-4385-4353-80b4-b19e643c5c9c", - "resource": { - "resourceType": "Claim", - "id": "4d3ba025-4385-4353-80b4-b19e643c5c9c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "prescription": { - "reference": "urn:uuid:a9493add-b2e9-46f9-bc58-6502eb98b460" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - } - ] - } - ], - "total": { - "value": 11.34, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:50484348-f8e8-438b-a79b-136d3960eb53", - "resource": { - "resourceType": "Claim", - "id": "50484348-f8e8-438b-a79b-136d3960eb53", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4c08e570-9d65-452f-a2e3-4f6784cfcf11", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4c08e570-9d65-452f-a2e3-4f6784cfcf11", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "50484348-f8e8-438b-a79b-136d3960eb53" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2016-08-01T21:20:11-07:00", - "end": "2017-08-01T21:20:11-07:00" - }, - "created": "2016-08-01T21:20:11-07:00", - "provider": { - "identifier": { - "value": "339a8b0f-de8f-3168-bfe3-89f8b4614840" - } - }, - "organization": { - "identifier": { - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:50484348-f8e8-438b-a79b-136d3960eb53" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:aa9df8bf-c94c-40a2-8483-fcfab7698c65" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3bd80262-f307-4c18-8198-72d23d3bf810" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-08-01T21:05:11-07:00", - "end": "2016-08-01T21:20:11-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431", - "resource": { - "resourceType": "Encounter", - "id": "32e3dc55-9e43-454a-b8bd-8636caed2431", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2016-12-14T20:05:11-08:00", - "end": "2016-12-14T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d4ae6708-d33f-4136-b1ec-fff488e3dfa6", - "resource": { - "resourceType": "Observation", - "id": "d4ae6708-d33f-4136-b1ec-fff488e3dfa6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 104.90, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd669a47-7e48-4fbc-bd5c-e64145f260bc", - "resource": { - "resourceType": "Observation", - "id": "cd669a47-7e48-4fbc-bd5c-e64145f260bc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f570a2d2-8fb3-4be4-9674-713995a72d6e", - "resource": { - "resourceType": "Observation", - "id": "f570a2d2-8fb3-4be4-9674-713995a72d6e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 16.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:338c794c-0240-421b-b8e8-854e6b3f6040", - "resource": { - "resourceType": "Observation", - "id": "338c794c-0240-421b-b8e8-854e6b3f6040", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 15.100, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:97ca8f54-1297-4a55-b251-0a44edf07957", - "resource": { - "resourceType": "Observation", - "id": "97ca8f54-1297-4a55-b251-0a44edf07957", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 38.379, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5c8f8ade-9882-486a-a65b-f8df5ef6d884", - "resource": { - "resourceType": "Observation", - "id": "5c8f8ade-9882-486a-a65b-f8df5ef6d884", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 118, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c502755-6247-4854-b958-6ac305837ce7", - "resource": { - "resourceType": "Observation", - "id": "7c502755-6247-4854-b958-6ac305837ce7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 92, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fabcc616-2fb9-4486-b7dc-b334292e1951", - "resource": { - "resourceType": "Observation", - "id": "fabcc616-2fb9-4486-b7dc-b334292e1951", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2ee4d489-1b7a-4a24-9a03-b8ca4813e965", - "resource": { - "resourceType": "Observation", - "id": "2ee4d489-1b7a-4a24-9a03-b8ca4813e965", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "effectiveDateTime": "2016-12-14T20:05:11-08:00", - "issued": "2016-12-14T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c46eaed1-1343-4862-9990-5f8f969bac48", - "resource": { - "resourceType": "Immunization", - "id": "c46eaed1-1343-4862-9990-5f8f969bac48", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "date": "2016-12-14T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:89be39a5-7a32-4e13-b5a8-914d56b69f44", - "resource": { - "resourceType": "Immunization", - "id": "89be39a5-7a32-4e13-b5a8-914d56b69f44", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "date": "2016-12-14T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:77fb8590-d460-4b5f-8729-c3f9ca2557c5", - "resource": { - "resourceType": "Immunization", - "id": "77fb8590-d460-4b5f-8729-c3f9ca2557c5", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "date": "2016-12-14T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:adae9219-f4b0-43bf-b48e-1c77d49ca55a", - "resource": { - "resourceType": "Immunization", - "id": "adae9219-f4b0-43bf-b48e-1c77d49ca55a", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "date": "2016-12-14T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:a41d2025-3ce6-48fa-9c88-60090d305cde", - "resource": { - "resourceType": "Immunization", - "id": "a41d2025-3ce6-48fa-9c88-60090d305cde", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - }, - "date": "2016-12-14T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8c2915cd-77a5-4b63-b6f4-d7461f1d761f", - "resource": { - "resourceType": "Claim", - "id": "8c2915cd-77a5-4b63-b6f4-d7461f1d761f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2016-12-14T20:05:11-08:00", - "end": "2016-12-14T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c46eaed1-1343-4862-9990-5f8f969bac48" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:89be39a5-7a32-4e13-b5a8-914d56b69f44" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:77fb8590-d460-4b5f-8729-c3f9ca2557c5" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:adae9219-f4b0-43bf-b48e-1c77d49ca55a" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:a41d2025-3ce6-48fa-9c88-60090d305cde" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:32e3dc55-9e43-454a-b8bd-8636caed2431" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:936fed72-50f6-4c0a-b60f-5d16b77c588f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "936fed72-50f6-4c0a-b60f-5d16b77c588f", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "8c2915cd-77a5-4b63-b6f4-d7461f1d761f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-12-14T20:20:11-08:00", - "end": "2017-12-14T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8", - "resource": { - "resourceType": "Encounter", - "id": "35981aed-9913-4e73-9dd0-3f60d0b33fb8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2017-12-20T20:05:11-08:00", - "end": "2017-12-20T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1bc1d55f-51d5-4f1f-be96-18967e7dbbdb", - "resource": { - "resourceType": "Observation", - "id": "1bc1d55f-51d5-4f1f-be96-18967e7dbbdb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 111.10, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b17a002f-243c-445a-b647-b9517a65bb0a", - "resource": { - "resourceType": "Observation", - "id": "b17a002f-243c-445a-b647-b9517a65bb0a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:38cbc1d5-1f2d-4418-9c45-494ed60b5d2b", - "resource": { - "resourceType": "Observation", - "id": "38cbc1d5-1f2d-4418-9c45-494ed60b5d2b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 19, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d123522c-61c4-48f6-9201-d786c56b56f6", - "resource": { - "resourceType": "Observation", - "id": "d123522c-61c4-48f6-9201-d786c56b56f6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 15.430, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:851d32e5-38c0-462c-b2fe-e5109360526f", - "resource": { - "resourceType": "Observation", - "id": "851d32e5-38c0-462c-b2fe-e5109360526f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 51.684, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:af271b9e-9056-4664-b0e3-d12788050fd1", - "resource": { - "resourceType": "Observation", - "id": "af271b9e-9056-4664-b0e3-d12788050fd1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1ae2627-9149-4100-93fb-2af4a2512b4b", - "resource": { - "resourceType": "Observation", - "id": "d1ae2627-9149-4100-93fb-2af4a2512b4b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 83, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a699695c-1228-4266-a933-2e33ecbb0f3a", - "resource": { - "resourceType": "Observation", - "id": "a699695c-1228-4266-a933-2e33ecbb0f3a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4ba136fd-edcb-4929-a1bd-95f8b7a60605", - "resource": { - "resourceType": "Observation", - "id": "4ba136fd-edcb-4929-a1bd-95f8b7a60605", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 10.106, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", - "resource": { - "resourceType": "Observation", - "id": "b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 5.4330, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:53a1d7e4-bcd1-490c-ba40-70f455b2cda8", - "resource": { - "resourceType": "Observation", - "id": "53a1d7e4-bcd1-490c-ba40-70f455b2cda8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 12.417, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9ee23889-a002-4b95-b887-cfd9233b62ad", - "resource": { - "resourceType": "Observation", - "id": "9ee23889-a002-4b95-b887-cfd9233b62ad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 44.338, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ddece709-c449-4ca4-85ff-755543487767", - "resource": { - "resourceType": "Observation", - "id": "ddece709-c449-4ca4-85ff-755543487767", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 93.280, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", - "resource": { - "resourceType": "Observation", - "id": "79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 31.997, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6d4da189-9300-49fc-ac5a-d709f8fa53f4", - "resource": { - "resourceType": "Observation", - "id": "6d4da189-9300-49fc-ac5a-d709f8fa53f4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 35.106, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:299455b2-3f69-4bce-acac-f170e788fd32", - "resource": { - "resourceType": "Observation", - "id": "299455b2-3f69-4bce-acac-f170e788fd32", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 39.407, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a69aa703-b799-4909-9488-e8b065f162c5", - "resource": { - "resourceType": "Observation", - "id": "a69aa703-b799-4909-9488-e8b065f162c5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 359.02, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebb6368f-d2ad-4b85-a279-3d49868b2105", - "resource": { - "resourceType": "Observation", - "id": "ebb6368f-d2ad-4b85-a279-3d49868b2105", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 325.39, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83aa45f2-ed79-4472-8481-f96de9e7b14a", - "resource": { - "resourceType": "Observation", - "id": "83aa45f2-ed79-4472-8481-f96de9e7b14a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueQuantity": { - "value": 9.4721, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a7cb848-3052-473f-a9f7-6b87f210bd34", - "resource": { - "resourceType": "Observation", - "id": "8a7cb848-3052-473f-a9f7-6b87f210bd34", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bdcec993-44b9-49ed-a9a5-46dc72a8e85b", - "resource": { - "resourceType": "Immunization", - "id": "bdcec993-44b9-49ed-a9a5-46dc72a8e85b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "date": "2017-12-20T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:50c78957-2341-4819-b93f-565dd19a828d", - "resource": { - "resourceType": "DiagnosticReport", - "id": "50c78957-2341-4819-b93f-565dd19a828d", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - }, - "effectiveDateTime": "2017-12-20T20:05:11-08:00", - "issued": "2017-12-20T20:05:11.927-08:00", - "result": [ - { - "reference": "urn:uuid:4ba136fd-edcb-4929-a1bd-95f8b7a60605", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:b7d10999-13ce-4bd2-ba9e-a3e2d98edc8f", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:53a1d7e4-bcd1-490c-ba40-70f455b2cda8", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:9ee23889-a002-4b95-b887-cfd9233b62ad", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:ddece709-c449-4ca4-85ff-755543487767", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:79d2ad09-bd86-4cc7-a3a1-cc5f20cd3d4c", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:6d4da189-9300-49fc-ac5a-d709f8fa53f4", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:299455b2-3f69-4bce-acac-f170e788fd32", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:a69aa703-b799-4909-9488-e8b065f162c5", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:ebb6368f-d2ad-4b85-a279-3d49868b2105", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:83aa45f2-ed79-4472-8481-f96de9e7b14a", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:f03e4ba9-9531-45ef-8e06-aae71d0f20b1", - "resource": { - "resourceType": "Claim", - "id": "f03e4ba9-9531-45ef-8e06-aae71d0f20b1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2017-12-20T20:05:11-08:00", - "end": "2017-12-20T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:bdcec993-44b9-49ed-a9a5-46dc72a8e85b" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:35981aed-9913-4e73-9dd0-3f60d0b33fb8" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a93c3d06-09db-4742-ba12-b11a89987343", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a93c3d06-09db-4742-ba12-b11a89987343", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f03e4ba9-9531-45ef-8e06-aae71d0f20b1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2017-12-20T20:20:11-08:00", - "end": "2018-12-20T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07", - "resource": { - "resourceType": "Encounter", - "id": "396ef73d-6551-43a5-a12a-97575e70ad07", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2018-03-27T21:05:11-07:00", - "end": "2018-03-27T21:20:11-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8", - "resource": { - "resourceType": "Condition", - "id": "fb53b877-fd5b-432a-80ff-6531ef2f9af8", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07" - }, - "onsetDateTime": "2018-03-27T21:05:11-07:00", - "abatementDateTime": "2018-04-10T21:05:11-07:00", - "assertedDate": "2018-03-27T21:05:11-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:67b9ab66-57a6-4ab0-917a-1240bea9d360", - "resource": { - "resourceType": "Claim", - "id": "67b9ab66-57a6-4ab0-917a-1240bea9d360", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2018-03-27T21:05:11-07:00", - "end": "2018-03-27T21:20:11-07:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:396ef73d-6551-43a5-a12a-97575e70ad07" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b56c52d9-3800-4877-a1e0-df6a585cabce", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b56c52d9-3800-4877-a1e0-df6a585cabce", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "67b9ab66-57a6-4ab0-917a-1240bea9d360" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-03-27T21:20:11-07:00", - "end": "2019-03-27T21:20:11-07:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:fb53b877-fd5b-432a-80ff-6531ef2f9af8" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622", - "resource": { - "resourceType": "Encounter", - "id": "8a6adcfd-e483-46a7-ad84-dbce1ddd9622", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2018-12-26T20:05:11-08:00", - "end": "2018-12-26T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:47631bea-a001-4c5b-9e53-ad82a5da19b0", - "resource": { - "resourceType": "Observation", - "id": "47631bea-a001-4c5b-9e53-ad82a5da19b0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 117.20, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5e6538c6-eedf-4f4d-95b2-9164222fe96e", - "resource": { - "resourceType": "Observation", - "id": "5e6538c6-eedf-4f4d-95b2-9164222fe96e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fb80f3f7-e5c4-4409-8236-cf99a4f476ee", - "resource": { - "resourceType": "Observation", - "id": "fb80f3f7-e5c4-4409-8236-cf99a4f476ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 21.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:522fc591-692c-4966-9e67-dba157bd2556", - "resource": { - "resourceType": "Observation", - "id": "522fc591-692c-4966-9e67-dba157bd2556", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 15.670, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f22c7dd0-5c8f-4f93-8dd3-7baa3c3f9326", - "resource": { - "resourceType": "Observation", - "id": "f22c7dd0-5c8f-4f93-8dd3-7baa3c3f9326", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 54.696, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e08de7f1-0ccd-4ee3-8f10-71a5f2a115d6", - "resource": { - "resourceType": "Observation", - "id": "e08de7f1-0ccd-4ee3-8f10-71a5f2a115d6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 85, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c137e52e-d15a-4e1d-a96e-5b797f831553", - "resource": { - "resourceType": "Observation", - "id": "c137e52e-d15a-4e1d-a96e-5b797f831553", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 92, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:da03a0e0-5031-4f15-8e01-77a38ddcf812", - "resource": { - "resourceType": "Observation", - "id": "da03a0e0-5031-4f15-8e01-77a38ddcf812", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a2113e92-289c-4ea0-a3ee-8b745963ea5f", - "resource": { - "resourceType": "Observation", - "id": "a2113e92-289c-4ea0-a3ee-8b745963ea5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "effectiveDateTime": "2018-12-26T20:05:11-08:00", - "issued": "2018-12-26T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:42dd3bca-4dbe-4a9a-ac20-0878d46beb84", - "resource": { - "resourceType": "Immunization", - "id": "42dd3bca-4dbe-4a9a-ac20-0878d46beb84", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - }, - "date": "2018-12-26T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1", - "resource": { - "resourceType": "Claim", - "id": "d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2018-12-26T20:05:11-08:00", - "end": "2018-12-26T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:42dd3bca-4dbe-4a9a-ac20-0878d46beb84" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:8a6adcfd-e483-46a7-ad84-dbce1ddd9622" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bf76dc6e-875f-4393-af68-704fe21afe0b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "bf76dc6e-875f-4393-af68-704fe21afe0b", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d7dc03b4-d1c6-49c9-ba49-1a6fc5f881d1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-12-26T20:20:11-08:00", - "end": "2019-12-26T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44", - "resource": { - "resourceType": "Encounter", - "id": "c233b803-3126-4dfa-a13a-0b729d571a44", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8" - } - } - ], - "period": { - "start": "2019-11-08T20:05:11-08:00", - "end": "2019-11-08T20:20:11-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24", - "resource": { - "resourceType": "Condition", - "id": "3e0d8841-6fc1-477f-9912-6db4d8720d24", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44" - }, - "onsetDateTime": "2019-11-08T20:05:11-08:00", - "abatementDateTime": "2019-11-22T20:05:11-08:00", - "assertedDate": "2019-11-08T20:05:11-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:528c31eb-2246-4b58-a9a2-07fb59b381d9", - "resource": { - "resourceType": "Claim", - "id": "528c31eb-2246-4b58-a9a2-07fb59b381d9", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2019-11-08T20:05:11-08:00", - "end": "2019-11-08T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c233b803-3126-4dfa-a13a-0b729d571a44" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4c7d6ac2-ae54-4152-bff0-9a09cc05ff51", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4c7d6ac2-ae54-4152-bff0-9a09cc05ff51", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "528c31eb-2246-4b58-a9a2-07fb59b381d9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-11-08T20:20:11-08:00", - "end": "2020-11-08T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "6a3782fa-7d6e-302f-bec5-695563b123a8" - } - }, - "organization": { - "identifier": { - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3e0d8841-6fc1-477f-9912-6db4d8720d24" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f", - "resource": { - "resourceType": "Encounter", - "id": "145b1c95-c364-4a78-a844-f5a67043542f", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0" - } - } - ], - "period": { - "start": "2020-01-01T20:05:11-08:00", - "end": "2020-01-01T20:20:11-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cf0c3103-f045-4c0f-b99e-c6be04e15933", - "resource": { - "resourceType": "Observation", - "id": "cf0c3103-f045-4c0f-b99e-c6be04e15933", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 123, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab2a369c-b257-4dd6-bd34-fd344763a07f", - "resource": { - "resourceType": "Observation", - "id": "ab2a369c-b257-4dd6-bd34-fd344763a07f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:97d3300a-8130-4fd5-be67-6edb249e749b", - "resource": { - "resourceType": "Observation", - "id": "97d3300a-8130-4fd5-be67-6edb249e749b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 24.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a0a9b1f3-201d-4f98-8b65-902c37885d5f", - "resource": { - "resourceType": "Observation", - "id": "a0a9b1f3-201d-4f98-8b65-902c37885d5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 15.970, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:363dee6e-a169-4914-a6bd-874ede37c24b", - "resource": { - "resourceType": "Observation", - "id": "363dee6e-a169-4914-a6bd-874ede37c24b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 55.212, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33fa8ad5-ae8f-4051-9a83-bdb1a39bdf9a", - "resource": { - "resourceType": "Observation", - "id": "33fa8ad5-ae8f-4051-9a83-bdb1a39bdf9a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 104, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab87697b-c351-4799-a1b9-e2448154dd2f", - "resource": { - "resourceType": "Observation", - "id": "ab87697b-c351-4799-a1b9-e2448154dd2f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 87, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9603d7c0-474c-4783-a3cd-2bb3e1173cc7", - "resource": { - "resourceType": "Observation", - "id": "9603d7c0-474c-4783-a3cd-2bb3e1173cc7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ba1da30b-a992-4066-ba97-7ccd956997df", - "resource": { - "resourceType": "Observation", - "id": "ba1da30b-a992-4066-ba97-7ccd956997df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "context": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "effectiveDateTime": "2020-01-01T20:05:11-08:00", - "issued": "2020-01-01T20:05:11.927-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7282f34b-e3a6-4037-81f0-e9d6eac7c6b3", - "resource": { - "resourceType": "Immunization", - "id": "7282f34b-e3a6-4037-81f0-e9d6eac7c6b3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "encounter": { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - }, - "date": "2020-01-01T20:05:11-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:337b8fa6-bdbd-4019-9aff-068fab01cdd2", - "resource": { - "resourceType": "Claim", - "id": "337b8fa6-bdbd-4019-9aff-068fab01cdd2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "start": "2020-01-01T20:05:11-08:00", - "end": "2020-01-01T20:20:11-08:00" - }, - "organization": { - "reference": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7282f34b-e3a6-4037-81f0-e9d6eac7c6b3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:145b1c95-c364-4a78-a844-f5a67043542f" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b2f2ebb2-a126-470e-bb1a-58511e87f5ec", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b2f2ebb2-a126-470e-bb1a-58511e87f5ec", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "337b8fa6-bdbd-4019-9aff-068fab01cdd2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:136f997d-0a94-4573-97da-a53b5060a612" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2020-01-01T20:20:11-08:00", - "end": "2021-01-01T20:20:11-08:00" - }, - "provider": { - "identifier": { - "value": "ba314750-701e-3370-ade0-270e178d80e0" - } - }, - "organization": { - "identifier": { - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json deleted file mode 100644 index 47d9b906b987..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Dannette613_Maggio310_74779846-85a4-4b26-9da4-414a5fec1aed.json +++ /dev/null @@ -1,18231 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed", - "resource": { - "resourceType": "Patient", - "id": "74779846-85a4-4b26-9da4-414a5fec1aed", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 1459817271572172624 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2106-3", - "display": "White" - } - }, - { - "url": "text", - "valueString": "White" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Verlie371 Reynolds644" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Easthampton", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.012217100297429643 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 10.98778289970257 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "74779846-85a4-4b26-9da4-414a5fec1aed" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "74779846-85a4-4b26-9da4-414a5fec1aed" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-42-4253" - } - ], - "name": [ - { - "use": "official", - "family": "Maggio310", - "given": [ - "Dannette613" - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-178-3601", - "use": "home" - } - ], - "gender": "female", - "birthDate": "2008-05-01", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.12613043432252 - }, - { - "url": "longitude", - "valueDecimal": -70.96006577849847 - } - ] - } - ], - "line": [ - "291 Thiel Spur" - ], - "city": "Whitman", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthBoolean": false, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", - "resource": { - "resourceType": "Organization", - "id": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SIGNATURE HEALTHCARE MEDICAL GROUP INC", - "telecom": [ - { - "system": "phone", - "value": "781-878-1700" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.119966999999995 - }, - { - "url": "longitude", - "valueDecimal": -70.957211 - } - ] - } - ], - "line": [ - "360 BROCKTON AVE" - ], - "city": "ABINGTON", - "state": "MA", - "postalCode": "02351-2186", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841", - "resource": { - "resourceType": "Practitioner", - "id": "3dde31b6-dbc2-394b-b2af-a3167d1f6841", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "81770" - } - ], - "active": true, - "name": [ - { - "family": "Ondricka197", - "given": [ - "Leisa54" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "360 BROCKTON AVE" - ], - "city": "ABINGTON", - "state": "MA", - "postalCode": "02351-2186", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927", - "resource": { - "resourceType": "Encounter", - "id": "e5ebcd42-3144-4f74-b410-0e924137a927", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2010-10-07T23:41:28-07:00", - "end": "2010-10-07T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:34a8192f-b7a2-4ffc-b836-a9e44941009d", - "resource": { - "resourceType": "Observation", - "id": "34a8192f-b7a2-4ffc-b836-a9e44941009d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 92.600, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a36d03b9-1920-424e-bd4a-fb5bebdda8a7", - "resource": { - "resourceType": "Observation", - "id": "a36d03b9-1920-424e-bd4a-fb5bebdda8a7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eff3d32b-907b-48be-b94d-6c26f1ea361e", - "resource": { - "resourceType": "Observation", - "id": "eff3d32b-907b-48be-b94d-6c26f1ea361e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 16.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:300c5bc5-ff4f-4bfc-b6c8-57cba5943212", - "resource": { - "resourceType": "Observation", - "id": "300c5bc5-ff4f-4bfc-b6c8-57cba5943212", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 97.537, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68f2b87e-7d26-4614-92c7-1654538bc329", - "resource": { - "resourceType": "Observation", - "id": "68f2b87e-7d26-4614-92c7-1654538bc329", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 49.260, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:332ca9b2-e15e-437c-86a7-d154b14bfda5", - "resource": { - "resourceType": "Observation", - "id": "332ca9b2-e15e-437c-86a7-d154b14bfda5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 18.990, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:491d72ab-c2b5-4971-b96c-316507e3e131", - "resource": { - "resourceType": "Observation", - "id": "491d72ab-c2b5-4971-b96c-316507e3e131", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.594, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c27dfbc6-bb82-436d-9b76-99da730928bd", - "resource": { - "resourceType": "Observation", - "id": "c27dfbc6-bb82-436d-9b76-99da730928bd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 70, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 135, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32c8cddb-fdc8-4e80-9415-557e16059689", - "resource": { - "resourceType": "Observation", - "id": "32c8cddb-fdc8-4e80-9415-557e16059689", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 83, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:74c5f339-8b2a-4ed9-92ea-3f3c76ac743f", - "resource": { - "resourceType": "Observation", - "id": "74c5f339-8b2a-4ed9-92ea-3f3c76ac743f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4cff6afb-2c40-4844-9f44-68f758947cea", - "resource": { - "resourceType": "Observation", - "id": "4cff6afb-2c40-4844-9f44-68f758947cea", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "effectiveDateTime": "2010-10-07T23:41:28-07:00", - "issued": "2010-10-07T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8fe227f5-c731-47bf-b581-f9f060994298", - "resource": { - "resourceType": "Immunization", - "id": "8fe227f5-c731-47bf-b581-f9f060994298", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "date": "2010-10-07T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:7c493b39-652c-43fd-b8f8-7bc2b13abd07", - "resource": { - "resourceType": "Immunization", - "id": "7c493b39-652c-43fd-b8f8-7bc2b13abd07", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "83", - "display": "Hep A, ped/adol, 2 dose" - } - ], - "text": "Hep A, ped/adol, 2 dose" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - }, - "date": "2010-10-07T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:01cc5efc-a50e-4e06-9c66-48479aa8ea5a", - "resource": { - "resourceType": "Claim", - "id": "01cc5efc-a50e-4e06-9c66-48479aa8ea5a", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2010-10-07T23:41:28-07:00", - "end": "2010-10-07T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8fe227f5-c731-47bf-b581-f9f060994298" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:7c493b39-652c-43fd-b8f8-7bc2b13abd07" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e5ebcd42-3144-4f74-b410-0e924137a927" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1c6b395b-df6d-484e-9531-6374352ca78c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1c6b395b-df6d-484e-9531-6374352ca78c", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "01cc5efc-a50e-4e06-9c66-48479aa8ea5a" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2010-10-07T23:56:28-07:00", - "end": "2011-10-07T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 224.83200000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", - "resource": { - "resourceType": "Organization", - "id": "c44f361c-2efb-3050-8f97-0354a12e2920", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "c44f361c-2efb-3050-8f97-0354a12e2920" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5089417000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", - "resource": { - "resourceType": "Practitioner", - "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "200" - } - ], - "active": true, - "name": [ - { - "family": "Padberg411", - "given": [ - "Nila48" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737", - "resource": { - "resourceType": "Encounter", - "id": "3309eef4-cf71-4b37-9270-1e5914307737", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" - } - } - ], - "period": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf", - "resource": { - "resourceType": "Condition", - "id": "72b92bb7-8725-46fc-a14c-611dda67bdcf", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "65363002", - "display": "Otitis media" - } - ], - "text": "Otitis media" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - }, - "onsetDateTime": "2010-12-17T22:41:28-08:00", - "abatementDateTime": "2011-04-07T23:41:28-07:00", - "assertedDate": "2010-12-17T22:41:28-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:58c42292-b70d-4472-929d-699e52f1c77c", - "resource": { - "resourceType": "MedicationRequest", - "id": "58c42292-b70d-4472-929d-699e52f1c77c", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "309097", - "display": "Cefuroxime 250 MG Oral Tablet" - } - ], - "text": "Cefuroxime 250 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - }, - "authoredOn": "2010-12-17T22:41:28-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" - }, - "onBehalfOf": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0705a7f2-dafa-4ad3-b326-b0c3962ecc25", - "resource": { - "resourceType": "Claim", - "id": "0705a7f2-dafa-4ad3-b326-b0c3962ecc25", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - }, - "prescription": { - "reference": "urn:uuid:58c42292-b70d-4472-929d-699e52f1c77c" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - } - ] - } - ], - "total": { - "value": 50.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4db09a9f-a5fa-41a1-a0eb-d35403e1d53d", - "resource": { - "resourceType": "MedicationRequest", - "id": "4db09a9f-a5fa-41a1-a0eb-d35403e1d53d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - }, - "authoredOn": "2010-12-17T22:41:28-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f" - }, - "onBehalfOf": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:9cb8f511-3e89-43e1-8874-9193b5873dd4", - "resource": { - "resourceType": "Claim", - "id": "9cb8f511-3e89-43e1-8874-9193b5873dd4", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - }, - "prescription": { - "reference": "urn:uuid:4db09a9f-a5fa-41a1-a0eb-d35403e1d53d" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - } - ] - } - ], - "total": { - "value": 7.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:20b8fb5e-4f8c-49b9-8abb-952ee2087bb6", - "resource": { - "resourceType": "Claim", - "id": "20b8fb5e-4f8c-49b9-8abb-952ee2087bb6", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b721dfed-3274-4146-8667-5483ee669030", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b721dfed-3274-4146-8667-5483ee669030", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "NO_INSURANCE" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "20b8fb5e-4f8c-49b9-8abb-952ee2087bb6" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2010-12-17T22:56:28-08:00", - "end": "2011-12-17T22:56:28-08:00" - }, - "created": "2010-12-17T22:56:28-08:00", - "provider": { - "identifier": { - "value": "d08d5908-40a4-3571-8c54-0cc029c8277f" - } - }, - "organization": { - "identifier": { - "value": "c44f361c-2efb-3050-8f97-0354a12e2920" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:20b8fb5e-4f8c-49b9-8abb-952ee2087bb6" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:72b92bb7-8725-46fc-a14c-611dda67bdcf" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3309eef4-cf71-4b37-9270-1e5914307737" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2010-12-17T22:41:28-08:00", - "end": "2010-12-17T22:56:28-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", - "resource": { - "resourceType": "Organization", - "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "GOOD SAMARITAN MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5084273000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "resource": { - "resourceType": "Practitioner", - "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "460" - } - ], - "active": true, - "name": [ - { - "family": "Fay398", - "given": [ - "Deneen201" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e", - "resource": { - "resourceType": "Encounter", - "id": "f3312988-276e-4486-83d8-d1d00f8fb52e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2011-02-21T22:41:28-08:00", - "end": "2011-02-21T22:56:28-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4", - "resource": { - "resourceType": "Condition", - "id": "f482fe43-8829-4943-ae76-59a8e03be4a4", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e" - }, - "onsetDateTime": "2011-02-21T22:41:28-08:00", - "abatementDateTime": "2011-02-28T22:41:28-08:00", - "assertedDate": "2011-02-21T22:41:28-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:9d51479d-a632-4b0c-a35c-ab75b010ef5c", - "resource": { - "resourceType": "Claim", - "id": "9d51479d-a632-4b0c-a35c-ab75b010ef5c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2011-02-21T22:41:28-08:00", - "end": "2011-02-21T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:f3312988-276e-4486-83d8-d1d00f8fb52e" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4124dba5-52f0-4064-aa61-ad5c311515c1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4124dba5-52f0-4064-aa61-ad5c311515c1", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9d51479d-a632-4b0c-a35c-ab75b010ef5c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2011-02-21T22:56:28-08:00", - "end": "2012-02-21T22:56:28-08:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:f482fe43-8829-4943-ae76-59a8e03be4a4" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88", - "resource": { - "resourceType": "Encounter", - "id": "9d0abe12-fe52-440b-be7c-a4f901b48e88", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2011-04-07T23:41:28-07:00", - "end": "2011-04-07T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d6dc6847-3da1-4a29-95ce-f43cfdf02dbb", - "resource": { - "resourceType": "Observation", - "id": "d6dc6847-3da1-4a29-95ce-f43cfdf02dbb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.800, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0ae4be2c-e803-4f78-992f-f19340441ca6", - "resource": { - "resourceType": "Observation", - "id": "0ae4be2c-e803-4f78-992f-f19340441ca6", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7a7e71ee-d58d-445e-8590-69f4b1f53552", - "resource": { - "resourceType": "Observation", - "id": "7a7e71ee-d58d-445e-8590-69f4b1f53552", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 17.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e85dca4a-2cbf-4e48-a7a3-6096fc9dc0c8", - "resource": { - "resourceType": "Observation", - "id": "e85dca4a-2cbf-4e48-a7a3-6096fc9dc0c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.701, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c4a8a86a-687f-43a2-9e3b-171d01a728dc", - "resource": { - "resourceType": "Observation", - "id": "c4a8a86a-687f-43a2-9e3b-171d01a728dc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 49.820, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed8a2cce-c94c-48cc-89dd-524b8cc46a5f", - "resource": { - "resourceType": "Observation", - "id": "ed8a2cce-c94c-48cc-89dd-524b8cc46a5f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 18.700, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6c442a5-657f-4939-81b9-db8096209d8f", - "resource": { - "resourceType": "Observation", - "id": "e6c442a5-657f-4939-81b9-db8096209d8f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.728, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:52b5eb5c-3f7d-439b-9563-48b50e200baa", - "resource": { - "resourceType": "Observation", - "id": "52b5eb5c-3f7d-439b-9563-48b50e200baa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 115, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1e36554-c799-40d2-a3e5-89d72a353ad8", - "resource": { - "resourceType": "Observation", - "id": "d1e36554-c799-40d2-a3e5-89d72a353ad8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 78, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c458ab22-4b31-4d4b-a688-a718c567c638", - "resource": { - "resourceType": "Observation", - "id": "c458ab22-4b31-4d4b-a688-a718c567c638", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1e9f2237-a788-4911-8683-8a34a715e498", - "resource": { - "resourceType": "Observation", - "id": "1e9f2237-a788-4911-8683-8a34a715e498", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "effectiveDateTime": "2011-04-07T23:41:28-07:00", - "issued": "2011-04-07T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dfb4acb6-d6aa-4d4e-9797-e594ed46ea34", - "resource": { - "resourceType": "Immunization", - "id": "dfb4acb6-d6aa-4d4e-9797-e594ed46ea34", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "date": "2011-04-07T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:58fa92cc-4318-4e1b-a2df-5d79549b1e4b", - "resource": { - "resourceType": "Immunization", - "id": "58fa92cc-4318-4e1b-a2df-5d79549b1e4b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - }, - "date": "2011-04-07T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:17d7579d-29f1-41a6-b57d-5449b24318f5", - "resource": { - "resourceType": "Claim", - "id": "17d7579d-29f1-41a6-b57d-5449b24318f5", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2011-04-07T23:41:28-07:00", - "end": "2011-04-07T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:dfb4acb6-d6aa-4d4e-9797-e594ed46ea34" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:58fa92cc-4318-4e1b-a2df-5d79549b1e4b" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:9d0abe12-fe52-440b-be7c-a4f901b48e88" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:717c1332-5233-4f94-8200-6a105acd9bca", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "717c1332-5233-4f94-8200-6a105acd9bca", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "17d7579d-29f1-41a6-b57d-5449b24318f5" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2011-04-07T23:56:28-07:00", - "end": "2012-04-07T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 224.83200000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4", - "resource": { - "resourceType": "Encounter", - "id": "2db4ee31-f666-4d2e-bc74-7fabd9ca59d4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2011-10-06T23:41:28-07:00", - "end": "2011-10-07T00:11:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:f09a50e7-cebd-45b5-bda1-cb5e34a46b1a", - "resource": { - "resourceType": "Observation", - "id": "f09a50e7-cebd-45b5-bda1-cb5e34a46b1a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 100.40, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:727f4b88-7381-41fa-a6d8-663112a6d8d1", - "resource": { - "resourceType": "Observation", - "id": "727f4b88-7381-41fa-a6d8-663112a6d8d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:735fc3c8-0e13-4832-a490-c703c645067a", - "resource": { - "resourceType": "Observation", - "id": "735fc3c8-0e13-4832-a490-c703c645067a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 18.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5d229eb-6cf7-412d-b68b-2afe39d83ad7", - "resource": { - "resourceType": "Observation", - "id": "a5d229eb-6cf7-412d-b68b-2afe39d83ad7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 97.324, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d9f635cb-b6a6-4c32-be65-371d7cde5186", - "resource": { - "resourceType": "Observation", - "id": "d9f635cb-b6a6-4c32-be65-371d7cde5186", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 49.870, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:939f3fe0-9125-4ffd-8d5a-d86dfc136ad7", - "resource": { - "resourceType": "Observation", - "id": "939f3fe0-9125-4ffd-8d5a-d86dfc136ad7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 18.160, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:16724756-bd97-4aef-945f-e1b4719d6d42", - "resource": { - "resourceType": "Observation", - "id": "16724756-bd97-4aef-945f-e1b4719d6d42", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 95.278, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9d09c1be-c425-4f5e-b86e-7584d9af2f4e", - "resource": { - "resourceType": "Observation", - "id": "9d09c1be-c425-4f5e-b86e-7584d9af2f4e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 109, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d77fc935-f60a-44d4-9224-5c3b097298fa", - "resource": { - "resourceType": "Observation", - "id": "d77fc935-f60a-44d4-9224-5c3b097298fa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 93, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f09965b0-154e-4160-b64d-045792c8b125", - "resource": { - "resourceType": "Observation", - "id": "f09965b0-154e-4160-b64d-045792c8b125", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:77fd6cae-569e-40d9-865a-e79094dda946", - "resource": { - "resourceType": "Observation", - "id": "77fd6cae-569e-40d9-865a-e79094dda946", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "effectiveDateTime": "2011-10-06T23:41:28-07:00", - "issued": "2011-10-06T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd2964ac-95bb-4828-9e15-94e45efaf6a6", - "resource": { - "resourceType": "Procedure", - "id": "cd2964ac-95bb-4828-9e15-94e45efaf6a6", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "performedPeriod": { - "start": "2011-10-06T23:41:28-07:00", - "end": "2011-10-06T23:56:28-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4088ae81-89e9-44c9-8122-cc344132f3cf", - "resource": { - "resourceType": "Immunization", - "id": "4088ae81-89e9-44c9-8122-cc344132f3cf", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "133", - "display": "Pneumococcal conjugate PCV 13" - } - ], - "text": "Pneumococcal conjugate PCV 13" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - }, - "date": "2011-10-06T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e2bb83a3-ad13-461b-9ec5-bbd4f475f736", - "resource": { - "resourceType": "Claim", - "id": "e2bb83a3-ad13-461b-9ec5-bbd4f475f736", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2011-10-06T23:41:28-07:00", - "end": "2011-10-07T00:11:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4088ae81-89e9-44c9-8122-cc344132f3cf" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:cd2964ac-95bb-4828-9e15-94e45efaf6a6" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2db4ee31-f666-4d2e-bc74-7fabd9ca59d4" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 670.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:43f21ab4-7c6a-4854-8c1f-ae6809c142ab", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "43f21ab4-7c6a-4854-8c1f-ae6809c142ab", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e2bb83a3-ad13-461b-9ec5-bbd4f475f736" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2011-10-07T00:11:28-07:00", - "end": "2012-10-07T00:11:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 134.052, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 536.208, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 670.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 670.26, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 648.624, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1", - "resource": { - "resourceType": "Encounter", - "id": "183e5799-9baa-4b23-bb26-e258f79c7bc1", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2011-11-02T23:41:28-07:00", - "end": "2011-11-02T23:56:28-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9", - "resource": { - "resourceType": "Condition", - "id": "7d48f9ed-889b-4cf3-aafd-69a9c4a587b9", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "444814009", - "display": "Viral sinusitis (disorder)" - } - ], - "text": "Viral sinusitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1" - }, - "onsetDateTime": "2011-11-02T23:41:28-07:00", - "abatementDateTime": "2011-11-23T22:41:28-08:00", - "assertedDate": "2011-11-02T23:41:28-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:bcdf9318-633d-4391-a3d8-ec701283f38c", - "resource": { - "resourceType": "Claim", - "id": "bcdf9318-633d-4391-a3d8-ec701283f38c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2011-11-02T23:41:28-07:00", - "end": "2011-11-02T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:183e5799-9baa-4b23-bb26-e258f79c7bc1" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4b900b30-5799-4f1d-9b04-7853a9e1550d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4b900b30-5799-4f1d-9b04-7853a9e1550d", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bcdf9318-633d-4391-a3d8-ec701283f38c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2011-11-02T23:56:28-07:00", - "end": "2012-11-02T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:7d48f9ed-889b-4cf3-aafd-69a9c4a587b9" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d", - "resource": { - "resourceType": "Encounter", - "id": "22a11f12-80f8-41bc-98eb-f68059caa86d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2012-03-11T23:41:28-07:00", - "end": "2012-03-12T00:41:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa", - "resource": { - "resourceType": "Condition", - "id": "3697931b-452d-4681-aa9a-1cf191ad96aa", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "62106007", - "display": "Concussion with no loss of consciousness" - } - ], - "text": "Concussion with no loss of consciousness" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" - }, - "onsetDateTime": "2012-03-11T23:41:28-07:00", - "abatementDateTime": "2012-05-10T23:41:28-07:00", - "assertedDate": "2012-03-11T23:41:28-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ca139494-e8a0-4147-bcfb-f1f05237c1ce", - "resource": { - "resourceType": "CarePlan", - "id": "ca139494-e8a0-4147-bcfb-f1f05237c1ce", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "47387005", - "display": "Head injury rehabilitation" - } - ], - "text": "Head injury rehabilitation" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" - }, - "period": { - "start": "2012-03-11T23:41:28-07:00", - "end": "2012-05-10T23:41:28-07:00" - }, - "addresses": [ - { - "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "183051005", - "display": "Recommendation to rest" - } - ], - "text": "Recommendation to rest" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "226138001", - "display": "Alcohol-free diet" - } - ], - "text": "Alcohol-free diet" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:d075f4b5-3bdf-4e3c-8d39-97f665898dec", - "resource": { - "resourceType": "Claim", - "id": "d075f4b5-3bdf-4e3c-8d39-97f665898dec", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2012-03-11T23:41:28-07:00", - "end": "2012-03-12T00:41:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:041ee73f-a2b7-485e-9c9a-a3ce61d61aeb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "041ee73f-a2b7-485e-9c9a-a3ce61d61aeb", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d075f4b5-3bdf-4e3c-8d39-97f665898dec" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2012-03-12T00:41:28-07:00", - "end": "2013-03-12T00:41:28-07:00" - }, - "created": "2012-03-12T00:41:28-07:00", - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:d075f4b5-3bdf-4e3c-8d39-97f665898dec" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:3697931b-452d-4681-aa9a-1cf191ad96aa" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2012-03-11T23:41:28-07:00", - "end": "2012-03-12T00:41:28-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:22a11f12-80f8-41bc-98eb-f68059caa86d" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2012-03-11T23:41:28-07:00", - "end": "2012-03-12T00:41:28-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c", - "resource": { - "resourceType": "Encounter", - "id": "2f6a8179-14c6-48c5-8e16-83b0c28c8e3c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2012-04-05T23:41:28-07:00", - "end": "2012-04-05T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:530fb772-2ae3-49c3-9584-8ece6fb95b78", - "resource": { - "resourceType": "Observation", - "id": "530fb772-2ae3-49c3-9584-8ece6fb95b78", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 104, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e9b47b26-3902-4d1c-b6bb-e7fb1f100450", - "resource": { - "resourceType": "Observation", - "id": "e9b47b26-3902-4d1c-b6bb-e7fb1f100450", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee398553-755c-4270-9e0a-6cbc2fde81b9", - "resource": { - "resourceType": "Observation", - "id": "ee398553-755c-4270-9e0a-6cbc2fde81b9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 19.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:584a6540-94e5-4631-b228-ecaa0c3940ee", - "resource": { - "resourceType": "Observation", - "id": "584a6540-94e5-4631-b228-ecaa0c3940ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "77606-2", - "display": "Weight-for-length Per age and sex" - } - ], - "text": "Weight-for-length Per age and sex" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 97.324, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8a70ed08-c562-4c20-b0a0-b9975c95a37c", - "resource": { - "resourceType": "Observation", - "id": "8a70ed08-c562-4c20-b0a0-b9975c95a37c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9843-4", - "display": "Head Occipital-frontal circumference" - } - ], - "text": "Head Occipital-frontal circumference" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 49.870, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3bb64db9-2e24-46fe-99ae-2be4476569b7", - "resource": { - "resourceType": "Observation", - "id": "3bb64db9-2e24-46fe-99ae-2be4476569b7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 17.790, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7195cbc2-53fe-4d0f-bf39-03573a548216", - "resource": { - "resourceType": "Observation", - "id": "7195cbc2-53fe-4d0f-bf39-03573a548216", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 93.806, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:860a47c4-6065-44bb-936e-c23cfcd14d91", - "resource": { - "resourceType": "Observation", - "id": "860a47c4-6065-44bb-936e-c23cfcd14d91", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:992ec7de-2724-4a42-931c-d844c5aafe12", - "resource": { - "resourceType": "Observation", - "id": "992ec7de-2724-4a42-931c-d844c5aafe12", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 69, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4eba63ad-1102-4a8e-860d-9625060c484d", - "resource": { - "resourceType": "Observation", - "id": "4eba63ad-1102-4a8e-860d-9625060c484d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed62ee67-1921-408e-ae5f-8210589c2530", - "resource": { - "resourceType": "Observation", - "id": "ed62ee67-1921-408e-ae5f-8210589c2530", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "effectiveDateTime": "2012-04-05T23:41:28-07:00", - "issued": "2012-04-05T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:63607712-e6ad-4ad0-8374-167292d2b732", - "resource": { - "resourceType": "Immunization", - "id": "63607712-e6ad-4ad0-8374-167292d2b732", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - }, - "date": "2012-04-05T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:44ff0e2e-37ea-46cc-90ca-7dec78716119", - "resource": { - "resourceType": "Claim", - "id": "44ff0e2e-37ea-46cc-90ca-7dec78716119", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2012-04-05T23:41:28-07:00", - "end": "2012-04-05T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:63607712-e6ad-4ad0-8374-167292d2b732" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2f6a8179-14c6-48c5-8e16-83b0c28c8e3c" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:01acdc68-c657-4980-b249-f74d2bde54e3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "01acdc68-c657-4980-b249-f74d2bde54e3", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "44ff0e2e-37ea-46cc-90ca-7dec78716119" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-04-05T23:56:28-07:00", - "end": "2013-04-05T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4a42aa41-c430-4bf7-8a27-b29bbe6d7d50", - "resource": { - "resourceType": "Encounter", - "id": "4a42aa41-c430-4bf7-8a27-b29bbe6d7d50", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2012-05-10T23:41:28-07:00", - "end": "2012-05-10T23:56:28-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "62106007", - "display": "Concussion with no loss of consciousness" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1b21ca69-4933-47fc-9362-d0506cf3d2ab", - "resource": { - "resourceType": "Claim", - "id": "1b21ca69-4933-47fc-9362-d0506cf3d2ab", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2012-05-10T23:41:28-07:00", - "end": "2012-05-10T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4a42aa41-c430-4bf7-8a27-b29bbe6d7d50" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:2ff6e675-a27b-48dc-9ed5-88c4202d0d34", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "2ff6e675-a27b-48dc-9ed5-88c4202d0d34", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1b21ca69-4933-47fc-9362-d0506cf3d2ab" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-05-10T23:56:28-07:00", - "end": "2013-05-10T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69", - "resource": { - "resourceType": "Encounter", - "id": "45c0b635-4235-491c-b1ff-9ffb7c7f8e69", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2013-04-11T23:41:28-07:00", - "end": "2013-04-11T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2b2973ea-c054-44fa-9572-e77b63e25874", - "resource": { - "resourceType": "Observation", - "id": "2b2973ea-c054-44fa-9572-e77b63e25874", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 111.30, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:917b89dc-e9ac-4faa-a652-17fbd36febb2", - "resource": { - "resourceType": "Observation", - "id": "917b89dc-e9ac-4faa-a652-17fbd36febb2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:52932aa8-82f9-4594-aefb-f6c68962dc58", - "resource": { - "resourceType": "Observation", - "id": "52932aa8-82f9-4594-aefb-f6c68962dc58", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 24.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7bc13f9d-54d8-40d2-b250-70d601baaec9", - "resource": { - "resourceType": "Observation", - "id": "7bc13f9d-54d8-40d2-b250-70d601baaec9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 19.770, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ebaeae02-06a2-4cbf-84c3-a6fedee5518c", - "resource": { - "resourceType": "Observation", - "id": "ebaeae02-06a2-4cbf-84c3-a6fedee5518c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 98.235, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ef996b79-7daf-4fe6-8657-dfc8d3b8825e", - "resource": { - "resourceType": "Observation", - "id": "ef996b79-7daf-4fe6-8657-dfc8d3b8825e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 118, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d530ecb0-4846-47e1-8fec-a8862790479a", - "resource": { - "resourceType": "Observation", - "id": "d530ecb0-4846-47e1-8fec-a8862790479a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 71, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee6c080f-aec2-4c79-b486-fb59a4a8a435", - "resource": { - "resourceType": "Observation", - "id": "ee6c080f-aec2-4c79-b486-fb59a4a8a435", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:80680d3c-a823-49d9-829b-f136f7bd140c", - "resource": { - "resourceType": "Observation", - "id": "80680d3c-a823-49d9-829b-f136f7bd140c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "effectiveDateTime": "2013-04-11T23:41:28-07:00", - "issued": "2013-04-11T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f198a0ac-e871-49f4-a716-67c260ce83b2", - "resource": { - "resourceType": "Immunization", - "id": "f198a0ac-e871-49f4-a716-67c260ce83b2", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "21", - "display": "varicella" - } - ], - "text": "varicella" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "date": "2013-04-11T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4155a7be-e200-4fd4-a5f0-a9d2dcfb977c", - "resource": { - "resourceType": "Immunization", - "id": "4155a7be-e200-4fd4-a5f0-a9d2dcfb977c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "10", - "display": "IPV" - } - ], - "text": "IPV" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "date": "2013-04-11T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543", - "resource": { - "resourceType": "Immunization", - "id": "8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "date": "2013-04-11T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:edee72cd-efe2-4fb4-abb6-dd19a1941511", - "resource": { - "resourceType": "Immunization", - "id": "edee72cd-efe2-4fb4-abb6-dd19a1941511", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "20", - "display": "DTaP" - } - ], - "text": "DTaP" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "date": "2013-04-11T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c4532309-bca7-4039-bca0-646214dd4157", - "resource": { - "resourceType": "Immunization", - "id": "c4532309-bca7-4039-bca0-646214dd4157", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "03", - "display": "MMR" - } - ], - "text": "MMR" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - }, - "date": "2013-04-11T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:2d47359e-0d62-46d6-8479-6bac149d5a13", - "resource": { - "resourceType": "Claim", - "id": "2d47359e-0d62-46d6-8479-6bac149d5a13", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2013-04-11T23:41:28-07:00", - "end": "2013-04-11T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f198a0ac-e871-49f4-a716-67c260ce83b2" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:4155a7be-e200-4fd4-a5f0-a9d2dcfb977c" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8ab13fe9-bcdf-4295-a4a6-1c77f6c4f543" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:edee72cd-efe2-4fb4-abb6-dd19a1941511" - } - }, - { - "sequence": 5, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c4532309-bca7-4039-bca0-646214dd4157" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:45c0b635-4235-491c-b1ff-9ffb7c7f8e69" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "informationLinkId": [ - 5 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5681015a-7192-4b17-a375-8ec036828a5e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5681015a-7192-4b17-a375-8ec036828a5e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2d47359e-0d62-46d6-8479-6bac149d5a13" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-04-11T23:56:28-07:00", - "end": "2014-04-11T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 562.08, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833", - "resource": { - "resourceType": "Encounter", - "id": "c19ef61d-add1-4fc9-808d-66c1dbe56833", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2014-04-17T23:41:28-07:00", - "end": "2014-04-17T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:2fa0933f-b5f9-4690-bc3a-940f6eec5020", - "resource": { - "resourceType": "Observation", - "id": "2fa0933f-b5f9-4690-bc3a-940f6eec5020", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 118.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8e3f3cad-428a-47e6-89e9-b136d6c32682", - "resource": { - "resourceType": "Observation", - "id": "8e3f3cad-428a-47e6-89e9-b136d6c32682", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dea4beea-7c95-470f-828d-0fc6b675beb4", - "resource": { - "resourceType": "Observation", - "id": "dea4beea-7c95-470f-828d-0fc6b675beb4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 27.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:db434f82-d9cf-4d8a-8db9-98214c182999", - "resource": { - "resourceType": "Observation", - "id": "db434f82-d9cf-4d8a-8db9-98214c182999", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 19.310, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:030cfeb0-62a9-4bd6-9385-5dac07a65a7f", - "resource": { - "resourceType": "Observation", - "id": "030cfeb0-62a9-4bd6-9385-5dac07a65a7f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.331, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:712c3100-3f61-4822-be80-fd18f9b3d10c", - "resource": { - "resourceType": "Observation", - "id": "712c3100-3f61-4822-be80-fd18f9b3d10c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 82, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1ee050f4-ee26-49bf-99fb-043f1c72489c", - "resource": { - "resourceType": "Observation", - "id": "1ee050f4-ee26-49bf-99fb-043f1c72489c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 65, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8017673c-788f-40e0-9771-ccb93138fe94", - "resource": { - "resourceType": "Observation", - "id": "8017673c-788f-40e0-9771-ccb93138fe94", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a868b9da-3b19-4a35-8342-b4c60a646323", - "resource": { - "resourceType": "Observation", - "id": "a868b9da-3b19-4a35-8342-b4c60a646323", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 9.7187, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", - "resource": { - "resourceType": "Observation", - "id": "af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 4.9436, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", - "resource": { - "resourceType": "Observation", - "id": "1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 15.663, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2d1da5b8-2b96-457d-9969-dde39bc851fe", - "resource": { - "resourceType": "Observation", - "id": "2d1da5b8-2b96-457d-9969-dde39bc851fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 40.869, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:048979bc-47b8-4357-a8d4-492328189f99", - "resource": { - "resourceType": "Observation", - "id": "048979bc-47b8-4357-a8d4-492328189f99", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 84.145, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", - "resource": { - "resourceType": "Observation", - "id": "b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 32.970, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1f03a4d1-8772-4091-a08d-400c085bd2b8", - "resource": { - "resourceType": "Observation", - "id": "1f03a4d1-8772-4091-a08d-400c085bd2b8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 34.981, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0af5dbbc-f03f-455d-a835-f58628306992", - "resource": { - "resourceType": "Observation", - "id": "0af5dbbc-f03f-455d-a835-f58628306992", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 44.016, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d437a5b7-45db-4c87-a645-255360935b43", - "resource": { - "resourceType": "Observation", - "id": "d437a5b7-45db-4c87-a645-255360935b43", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 156.02, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e29e77b5-235d-4cfb-a691-e9586c37fabd", - "resource": { - "resourceType": "Observation", - "id": "e29e77b5-235d-4cfb-a691-e9586c37fabd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 448.52, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e9cd132f-c636-4791-95a7-ed45ca72a048", - "resource": { - "resourceType": "Observation", - "id": "e9cd132f-c636-4791-95a7-ed45ca72a048", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueQuantity": { - "value": 10.855, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d815087-aeac-46f5-947d-d77492ee4982", - "resource": { - "resourceType": "Observation", - "id": "1d815087-aeac-46f5-947d-d77492ee4982", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82c07d2a-8d8d-49e0-ac46-e0aa65a715ef", - "resource": { - "resourceType": "Immunization", - "id": "82c07d2a-8d8d-49e0-ac46-e0aa65a715ef", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "date": "2014-04-17T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4082a6bf-e557-4084-95f9-63d1308c8cf8", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4082a6bf-e557-4084-95f9-63d1308c8cf8", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - }, - "effectiveDateTime": "2014-04-17T23:41:28-07:00", - "issued": "2014-04-17T23:41:28.175-07:00", - "result": [ - { - "reference": "urn:uuid:a868b9da-3b19-4a35-8342-b4c60a646323", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:af6eee5b-40c4-4791-9a20-9a6f0ca6cdf7", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:1f5b893f-d2f7-41bb-acaa-4648ce12ab5a", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:2d1da5b8-2b96-457d-9969-dde39bc851fe", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:048979bc-47b8-4357-a8d4-492328189f99", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:b3f5ff57-9b5f-46a7-9da3-5686bf5ca0f5", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:1f03a4d1-8772-4091-a08d-400c085bd2b8", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:0af5dbbc-f03f-455d-a835-f58628306992", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:d437a5b7-45db-4c87-a645-255360935b43", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:e29e77b5-235d-4cfb-a691-e9586c37fabd", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:e9cd132f-c636-4791-95a7-ed45ca72a048", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:0538b525-e131-43da-8175-11faa1dded37", - "resource": { - "resourceType": "Claim", - "id": "0538b525-e131-43da-8175-11faa1dded37", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2014-04-17T23:41:28-07:00", - "end": "2014-04-17T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:82c07d2a-8d8d-49e0-ac46-e0aa65a715ef" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c19ef61d-add1-4fc9-808d-66c1dbe56833" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4e7d7291-ab0d-4552-9cf5-db775b5ae812", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4e7d7291-ab0d-4552-9cf5-db775b5ae812", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "0538b525-e131-43da-8175-11faa1dded37" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-04-17T23:56:28-07:00", - "end": "2015-04-17T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527", - "resource": { - "resourceType": "Encounter", - "id": "fa0b0c4d-02fe-487e-9d31-20cdc2f7c527", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2015-03-28T23:41:28-07:00", - "end": "2015-03-29T00:11:28-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819", - "resource": { - "resourceType": "Condition", - "id": "461d127e-9029-4afd-b821-b35c230b6819", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" - }, - "onsetDateTime": "2015-03-28T23:41:28-07:00", - "abatementDateTime": "2015-04-06T23:41:28-07:00", - "assertedDate": "2015-03-28T23:41:28-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:095fb91f-d9a4-4b33-bd42-337d14b3488e", - "resource": { - "resourceType": "Observation", - "id": "095fb91f-d9a4-4b33-bd42-337d14b3488e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" - }, - "effectiveDateTime": "2015-03-28T23:41:28-07:00", - "issued": "2015-03-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 37.213, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5653153-cf11-41b0-b578-746ebdcff635", - "resource": { - "resourceType": "Procedure", - "id": "a5653153-cf11-41b0-b578-746ebdcff635", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "117015009", - "display": "Throat culture (procedure)" - } - ], - "text": "Throat culture (procedure)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" - }, - "performedPeriod": { - "start": "2015-03-28T23:41:28-07:00", - "end": "2015-03-28T23:56:28-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819", - "display": "Acute viral pharyngitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9d45adf5-e694-4b1c-9ad3-978e2e992ed7", - "resource": { - "resourceType": "Claim", - "id": "9d45adf5-e694-4b1c-9ad3-978e2e992ed7", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2015-03-28T23:41:28-07:00", - "end": "2015-03-29T00:11:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a5653153-cf11-41b0-b578-746ebdcff635" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:fa0b0c4d-02fe-487e-9d31-20cdc2f7c527" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "net": { - "value": 4036.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:87d56cb6-5891-4a0d-852c-c759ac33d286", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "87d56cb6-5891-4a0d-852c-c759ac33d286", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9d45adf5-e694-4b1c-9ad3-978e2e992ed7" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-03-29T00:11:28-07:00", - "end": "2016-03-29T00:11:28-07:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:461d127e-9029-4afd-b821-b35c230b6819" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "117015009" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 807.244, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3228.976, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 4036.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 4036.22, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 3228.976, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5", - "resource": { - "resourceType": "Encounter", - "id": "cd225c5f-ea00-47da-aea5-f0672f13d3e5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2015-04-23T23:41:28-07:00", - "end": "2015-04-23T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5c275f33-5da3-4e8a-b8e1-7154a1ef6cf5", - "resource": { - "resourceType": "Observation", - "id": "5c275f33-5da3-4e8a-b8e1-7154a1ef6cf5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 125.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a3441f1-7561-46ce-8ede-8ee7d5df46c2", - "resource": { - "resourceType": "Observation", - "id": "1a3441f1-7561-46ce-8ede-8ee7d5df46c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:340d4f4f-afc4-4399-8efe-a070abf1232c", - "resource": { - "resourceType": "Observation", - "id": "340d4f4f-afc4-4399-8efe-a070abf1232c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 29.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:75a015c9-c028-4561-bcdd-aa168eb8bc60", - "resource": { - "resourceType": "Observation", - "id": "75a015c9-c028-4561-bcdd-aa168eb8bc60", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 18.710, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0e4f6eef-b361-4d4b-8b47-c862c2023862", - "resource": { - "resourceType": "Observation", - "id": "0e4f6eef-b361-4d4b-8b47-c862c2023862", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 92.067, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e8bdb47c-54d0-44b9-863a-76a19cb07442", - "resource": { - "resourceType": "Observation", - "id": "e8bdb47c-54d0-44b9-863a-76a19cb07442", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 77, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 133, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8bda595-7fc7-4f89-b4d4-384ad4928fb2", - "resource": { - "resourceType": "Observation", - "id": "b8bda595-7fc7-4f89-b4d4-384ad4928fb2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 71, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a60af7cd-62cf-4563-84bf-36b3506bfb17", - "resource": { - "resourceType": "Observation", - "id": "a60af7cd-62cf-4563-84bf-36b3506bfb17", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueQuantity": { - "value": 12, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f0dcf832-bd29-49ac-b4f6-1832349653e9", - "resource": { - "resourceType": "Observation", - "id": "f0dcf832-bd29-49ac-b4f6-1832349653e9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "effectiveDateTime": "2015-04-23T23:41:28-07:00", - "issued": "2015-04-23T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:15451914-2ac0-445d-9bea-55c407260c8f", - "resource": { - "resourceType": "Immunization", - "id": "15451914-2ac0-445d-9bea-55c407260c8f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - }, - "date": "2015-04-23T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:c410c39e-37e5-4884-828e-9e30fc01f547", - "resource": { - "resourceType": "Claim", - "id": "c410c39e-37e5-4884-828e-9e30fc01f547", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2015-04-23T23:41:28-07:00", - "end": "2015-04-23T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:15451914-2ac0-445d-9bea-55c407260c8f" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:cd225c5f-ea00-47da-aea5-f0672f13d3e5" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:bc77430b-e5cd-45c1-87eb-4664b60b49c9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "bc77430b-e5cd-45c1-87eb-4664b60b49c9", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c410c39e-37e5-4884-828e-9e30fc01f547" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-04-23T23:56:28-07:00", - "end": "2016-04-23T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de", - "resource": { - "resourceType": "Encounter", - "id": "1d9f5d50-0b15-423e-b7c8-7f76dad7a5de", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2016-04-28T23:41:28-07:00", - "end": "2016-04-29T00:11:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:beb1f62e-15cf-472a-8900-71908c69f007", - "resource": { - "resourceType": "Observation", - "id": "beb1f62e-15cf-472a-8900-71908c69f007", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 132.20, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:031335b4-b74a-4313-a3a6-6ec46092f24d", - "resource": { - "resourceType": "Observation", - "id": "031335b4-b74a-4313-a3a6-6ec46092f24d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9626ab5e-78a6-4a10-9a67-019efb780f1a", - "resource": { - "resourceType": "Observation", - "id": "9626ab5e-78a6-4a10-9a67-019efb780f1a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 41.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:47232ee4-8643-4c87-970a-5bc5993a6fc4", - "resource": { - "resourceType": "Observation", - "id": "47232ee4-8643-4c87-970a-5bc5993a6fc4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 23.670, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ab9bdfe9-2aff-4e60-b5cd-dccb3ec27990", - "resource": { - "resourceType": "Observation", - "id": "ab9bdfe9-2aff-4e60-b5cd-dccb3ec27990", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 98.383, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:19129d9f-0b61-42a9-b7a9-f65558e77e65", - "resource": { - "resourceType": "Observation", - "id": "19129d9f-0b61-42a9-b7a9-f65558e77e65", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 127, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:40b2e56f-566f-469c-9e54-d95ab4a6bc92", - "resource": { - "resourceType": "Observation", - "id": "40b2e56f-566f-469c-9e54-d95ab4a6bc92", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 78, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:69ee5599-3732-4333-8945-59467ca787a0", - "resource": { - "resourceType": "Observation", - "id": "69ee5599-3732-4333-8945-59467ca787a0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a99da671-1992-483e-95c7-5528ead3e793", - "resource": { - "resourceType": "Observation", - "id": "a99da671-1992-483e-95c7-5528ead3e793", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "effectiveDateTime": "2016-04-28T23:41:28-07:00", - "issued": "2016-04-28T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:096db481-b816-412d-ad68-424a0ac417e6", - "resource": { - "resourceType": "Procedure", - "id": "096db481-b816-412d-ad68-424a0ac417e6", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "performedPeriod": { - "start": "2016-04-28T23:41:28-07:00", - "end": "2016-04-28T23:56:28-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:65c0a039-6927-4754-ad69-9efb59cb2ba3", - "resource": { - "resourceType": "Immunization", - "id": "65c0a039-6927-4754-ad69-9efb59cb2ba3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - }, - "date": "2016-04-28T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2", - "resource": { - "resourceType": "Claim", - "id": "57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2016-04-28T23:41:28-07:00", - "end": "2016-04-29T00:11:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:65c0a039-6927-4754-ad69-9efb59cb2ba3" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:096db481-b816-412d-ad68-424a0ac417e6" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1d9f5d50-0b15-423e-b7c8-7f76dad7a5de" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 552.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e02bbf46-7a92-4921-8a2c-a9d6d3825093", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e02bbf46-7a92-4921-8a2c-a9d6d3825093", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "57c68025-1ee6-47d7-9dc6-0fd2bb8d53b2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-04-29T00:11:28-07:00", - "end": "2017-04-29T00:11:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 110.474, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 441.896, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 552.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 552.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 554.312, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65", - "resource": { - "resourceType": "Encounter", - "id": "01b37d9a-3c03-4380-9c0e-2161b13c9f65", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2016-11-30T22:41:28-08:00", - "end": "2016-11-30T22:56:28-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5", - "resource": { - "resourceType": "Condition", - "id": "0819f7ff-bcf6-4fd5-bef2-003de0b4cad5", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "43878008", - "display": "Streptococcal sore throat (disorder)" - } - ], - "text": "Streptococcal sore throat (disorder)" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" - }, - "onsetDateTime": "2016-11-30T22:41:28-08:00", - "abatementDateTime": "2016-12-11T22:41:28-08:00", - "assertedDate": "2016-11-30T22:41:28-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:ccb86374-8d06-4763-8d81-5f5f38964b16", - "resource": { - "resourceType": "Observation", - "id": "ccb86374-8d06-4763-8d81-5f5f38964b16", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" - }, - "effectiveDateTime": "2016-11-30T22:41:28-08:00", - "issued": "2016-11-30T22:41:28.175-08:00", - "valueQuantity": { - "value": 38.853, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4b86d0b2-5bcd-40f3-b375-72e8bd3b9836", - "resource": { - "resourceType": "MedicationRequest", - "id": "4b86d0b2-5bcd-40f3-b375-72e8bd3b9836", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "834061", - "display": "Penicillin V Potassium 250 MG Oral Tablet" - } - ], - "text": "Penicillin V Potassium 250 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" - }, - "authoredOn": "2016-11-30T22:41:28-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - }, - "onBehalfOf": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:25860c6e-63d3-44ad-a475-ec6ea05f6ae8", - "resource": { - "resourceType": "Claim", - "id": "25860c6e-63d3-44ad-a475-ec6ea05f6ae8", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2016-11-30T22:41:28-08:00", - "end": "2016-11-30T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "prescription": { - "reference": "urn:uuid:4b86d0b2-5bcd-40f3-b375-72e8bd3b9836" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" - } - ] - } - ], - "total": { - "value": 12.27, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6244ee17-361f-40fc-9de8-c2619da63404", - "resource": { - "resourceType": "Claim", - "id": "6244ee17-361f-40fc-9de8-c2619da63404", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2016-11-30T22:41:28-08:00", - "end": "2016-11-30T22:56:28-08:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:01b37d9a-3c03-4380-9c0e-2161b13c9f65" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5f22491b-f8ec-489a-997f-aff564a1e9a9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5f22491b-f8ec-489a-997f-aff564a1e9a9", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "6244ee17-361f-40fc-9de8-c2619da63404" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-11-30T22:56:28-08:00", - "end": "2017-11-30T22:56:28-08:00" - }, - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:0819f7ff-bcf6-4fd5-bef2-003de0b4cad5" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63", - "resource": { - "resourceType": "Encounter", - "id": "d791bca0-64d4-4169-8b1c-26f821f1fe63", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2017-05-04T23:41:28-07:00", - "end": "2017-05-04T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3f33d17c-0ea5-43b0-87e6-e2a269a9aa82", - "resource": { - "resourceType": "Observation", - "id": "3f33d17c-0ea5-43b0-87e6-e2a269a9aa82", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 138.30, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d9ee8452-e3c4-4197-9d4c-9924bacf089b", - "resource": { - "resourceType": "Observation", - "id": "d9ee8452-e3c4-4197-9d4c-9924bacf089b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:36bbe7db-206f-4b9b-930d-77ed432cbcca", - "resource": { - "resourceType": "Observation", - "id": "36bbe7db-206f-4b9b-930d-77ed432cbcca", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 46.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bd5525f4-a346-4675-9f56-db1337711563", - "resource": { - "resourceType": "Observation", - "id": "bd5525f4-a346-4675-9f56-db1337711563", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 24.120, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:befd064f-b53a-4239-bb73-6f8a797a0750", - "resource": { - "resourceType": "Observation", - "id": "befd064f-b53a-4239-bb73-6f8a797a0750", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 97.701, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6aee833a-c46c-49d5-8271-e664bcf1a778", - "resource": { - "resourceType": "Observation", - "id": "6aee833a-c46c-49d5-8271-e664bcf1a778", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 125, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ffc79ca3-8ccf-4dfe-bcf1-16c5884131ad", - "resource": { - "resourceType": "Observation", - "id": "ffc79ca3-8ccf-4dfe-bcf1-16c5884131ad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 76, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b2711b15-1375-41d5-bf47-5a1f6227122a", - "resource": { - "resourceType": "Observation", - "id": "b2711b15-1375-41d5-bf47-5a1f6227122a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:eee4571d-d19f-4fa9-b7ee-f2039023da34", - "resource": { - "resourceType": "Observation", - "id": "eee4571d-d19f-4fa9-b7ee-f2039023da34", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "effectiveDateTime": "2017-05-04T23:41:28-07:00", - "issued": "2017-05-04T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:511d15bf-031e-42ca-8f59-4534d2feb31f", - "resource": { - "resourceType": "Immunization", - "id": "511d15bf-031e-42ca-8f59-4534d2feb31f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - }, - "date": "2017-05-04T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b3a8886b-db9a-4761-ab68-629ce30e90eb", - "resource": { - "resourceType": "Claim", - "id": "b3a8886b-db9a-4761-ab68-629ce30e90eb", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2017-05-04T23:41:28-07:00", - "end": "2017-05-04T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:511d15bf-031e-42ca-8f59-4534d2feb31f" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:d791bca0-64d4-4169-8b1c-26f821f1fe63" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:be175f10-ba2d-4ce8-bdc2-1bd62979b84f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "be175f10-ba2d-4ce8-bdc2-1bd62979b84f", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b3a8886b-db9a-4761-ab68-629ce30e90eb" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2017-05-04T23:56:28-07:00", - "end": "2018-05-04T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed", - "resource": { - "resourceType": "Encounter", - "id": "bd967ad1-5a15-4686-9618-fded1705f3ed", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2018-05-10T23:41:28-07:00", - "end": "2018-05-10T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:80946ae4-c740-4dd4-859f-63fceab53d5d", - "resource": { - "resourceType": "Observation", - "id": "80946ae4-c740-4dd4-859f-63fceab53d5d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 143.70, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1724ca7a-5f9c-4cb3-83f1-a1a561ba5d6e", - "resource": { - "resourceType": "Observation", - "id": "1724ca7a-5f9c-4cb3-83f1-a1a561ba5d6e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:83eff34f-9189-4c30-b328-090fcfe6bba3", - "resource": { - "resourceType": "Observation", - "id": "83eff34f-9189-4c30-b328-090fcfe6bba3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 49.200, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99b0abc7-ec89-4299-b861-73d79626fe37", - "resource": { - "resourceType": "Observation", - "id": "99b0abc7-ec89-4299-b861-73d79626fe37", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 23.830, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5387c90c-a7e0-4836-b7e6-bc8821ea907d", - "resource": { - "resourceType": "Observation", - "id": "5387c90c-a7e0-4836-b7e6-bc8821ea907d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 96.200, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f9b31e6f-a59c-4073-a13c-27af3b4b5215", - "resource": { - "resourceType": "Observation", - "id": "f9b31e6f-a59c-4073-a13c-27af3b4b5215", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 122, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3bfdc40d-1ab0-4f94-84f8-81a44e651471", - "resource": { - "resourceType": "Observation", - "id": "3bfdc40d-1ab0-4f94-84f8-81a44e651471", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 85, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:61acc741-5ea8-450e-9d10-207e44579856", - "resource": { - "resourceType": "Observation", - "id": "61acc741-5ea8-450e-9d10-207e44579856", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:42f62174-98bc-4624-8a12-f321654f15ff", - "resource": { - "resourceType": "Observation", - "id": "42f62174-98bc-4624-8a12-f321654f15ff", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "effectiveDateTime": "2018-05-10T23:41:28-07:00", - "issued": "2018-05-10T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32c5f73e-c88e-49b0-8cf9-14af8aee1ab3", - "resource": { - "resourceType": "Immunization", - "id": "32c5f73e-c88e-49b0-8cf9-14af8aee1ab3", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - }, - "date": "2018-05-10T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:1a6aa059-53e1-4cff-b8e8-5243b001997c", - "resource": { - "resourceType": "Claim", - "id": "1a6aa059-53e1-4cff-b8e8-5243b001997c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2018-05-10T23:41:28-07:00", - "end": "2018-05-10T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:32c5f73e-c88e-49b0-8cf9-14af8aee1ab3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:bd967ad1-5a15-4686-9618-fded1705f3ed" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c8122e8b-4a3f-45a9-8e01-e52123646895", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c8122e8b-4a3f-45a9-8e01-e52123646895", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "1a6aa059-53e1-4cff-b8e8-5243b001997c" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-05-10T23:56:28-07:00", - "end": "2019-05-10T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252", - "resource": { - "resourceType": "Encounter", - "id": "93415003-cc07-4544-8c69-34c3a9fad252", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - } - ], - "period": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890", - "resource": { - "resourceType": "Condition", - "id": "01e122c3-3db5-49ee-b256-9cbe928b9890", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "283371005", - "display": "Laceration of forearm" - } - ], - "text": "Laceration of forearm" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - }, - "onsetDateTime": "2018-06-08T23:41:28-07:00", - "abatementDateTime": "2018-06-22T23:41:28-07:00", - "assertedDate": "2018-06-08T23:41:28-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:5baa99a7-6e14-4553-9a0e-7d425bc4b6cf", - "resource": { - "resourceType": "Procedure", - "id": "5baa99a7-6e14-4553-9a0e-7d425bc4b6cf", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "288086009", - "display": "Suture open wound" - } - ], - "text": "Suture open wound" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - }, - "performedPeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-08T23:53:28-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890", - "display": "Laceration of forearm" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4573a9d7-d985-4d8c-815b-b8ce3fe66769", - "resource": { - "resourceType": "MedicationRequest", - "id": "4573a9d7-d985-4d8c-815b-b8ce3fe66769", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313820", - "display": "Acetaminophen 160 MG Chewable Tablet" - } - ], - "text": "Acetaminophen 160 MG Chewable Tablet" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - }, - "authoredOn": "2018-06-08T23:41:28-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1" - }, - "onBehalfOf": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:f2e15c0e-af89-4a65-9ce2-9e642f12416e", - "resource": { - "resourceType": "Claim", - "id": "f2e15c0e-af89-4a65-9ce2-9e642f12416e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "prescription": { - "reference": "urn:uuid:4573a9d7-d985-4d8c-815b-b8ce3fe66769" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - } - ] - } - ], - "total": { - "value": 3.2, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:500c7860-eb6d-4764-840b-c10727e1a251", - "resource": { - "resourceType": "CarePlan", - "id": "500c7860-eb6d-4764-840b-c10727e1a251", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225358003", - "display": "Wound care" - } - ], - "text": "Wound care" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - }, - "period": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-22T23:41:28-07:00" - }, - "addresses": [ - { - "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385949008", - "display": "Dressing change management" - } - ], - "text": "Dressing change management" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439830001", - "display": "Behavior to prevent infection" - } - ], - "text": "Behavior to prevent infection" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:78afde0b-9dd7-4f69-af1f-91ce8eb567ec", - "resource": { - "resourceType": "Claim", - "id": "78afde0b-9dd7-4f69-af1f-91ce8eb567ec", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "organization": { - "reference": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:5baa99a7-6e14-4553-9a0e-7d425bc4b6cf" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "288086009" - } - ] - }, - "net": { - "value": 11608.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7286ec4f-02da-44da-9f1f-0e6ef6e332e1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7286ec4f-02da-44da-9f1f-0e6ef6e332e1", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "78afde0b-9dd7-4f69-af1f-91ce8eb567ec" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2018-06-09T00:53:28-07:00", - "end": "2019-06-09T00:53:28-07:00" - }, - "created": "2018-06-09T00:53:28-07:00", - "provider": { - "identifier": { - "value": "27fac077-3105-3983-8b0f-cc4f30f9e7c1" - } - }, - "organization": { - "identifier": { - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:78afde0b-9dd7-4f69-af1f-91ce8eb567ec" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:01e122c3-3db5-49ee-b256-9cbe928b9890" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:93415003-cc07-4544-8c69-34c3a9fad252" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "288086009" - } - ] - }, - "servicedPeriod": { - "start": "2018-06-08T23:41:28-07:00", - "end": "2018-06-09T00:53:28-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "net": { - "value": 11608.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2321.746, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9286.984, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11608.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11608.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 9286.984, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e", - "resource": { - "resourceType": "Encounter", - "id": "eceec331-ba5c-4b19-b3e1-51c1faabf52e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - } - ], - "period": { - "start": "2019-05-16T23:41:28-07:00", - "end": "2019-05-16T23:56:28-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9c0aa340-be98-44b2-893f-64f2b106ae6c", - "resource": { - "resourceType": "Observation", - "id": "9c0aa340-be98-44b2-893f-64f2b106ae6c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 150.20, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b4afdce7-64fc-4214-9000-46763f503d55", - "resource": { - "resourceType": "Observation", - "id": "b4afdce7-64fc-4214-9000-46763f503d55", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a073353e-f81f-4b21-bbdf-2978a4b6f2bf", - "resource": { - "resourceType": "Observation", - "id": "a073353e-f81f-4b21-bbdf-2978a4b6f2bf", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 47.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1d15fa79-f80f-4f17-8a7a-4ecf0144e055", - "resource": { - "resourceType": "Observation", - "id": "1d15fa79-f80f-4f17-8a7a-4ecf0144e055", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 21.040, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:df2a62f1-a58e-4f82-b77d-83ab9a0e46ca", - "resource": { - "resourceType": "Observation", - "id": "df2a62f1-a58e-4f82-b77d-83ab9a0e46ca", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 85.914, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a8918824-a5b5-4196-be99-711cadd9c35e", - "resource": { - "resourceType": "Observation", - "id": "a8918824-a5b5-4196-be99-711cadd9c35e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 129, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0c5fb298-fca5-42c7-95a1-dbca00e3c7d8", - "resource": { - "resourceType": "Observation", - "id": "0c5fb298-fca5-42c7-95a1-dbca00e3c7d8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 68, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:35cb4ea6-e2fd-47ce-8a5f-e515f39398b5", - "resource": { - "resourceType": "Observation", - "id": "35cb4ea6-e2fd-47ce-8a5f-e515f39398b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9cfb44ff-4922-493b-9e66-fa1b6f269161", - "resource": { - "resourceType": "Observation", - "id": "9cfb44ff-4922-493b-9e66-fa1b6f269161", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 6.8759, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0033e9f6-3b36-4a23-8839-d727c750fa93", - "resource": { - "resourceType": "Observation", - "id": "0033e9f6-3b36-4a23-8839-d727c750fa93", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 4.0275, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3c46285f-0b7b-4bc6-be32-a2f9db82cca2", - "resource": { - "resourceType": "Observation", - "id": "3c46285f-0b7b-4bc6-be32-a2f9db82cca2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 14.612, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", - "resource": { - "resourceType": "Observation", - "id": "6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 47.912, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b589fc38-f806-41e6-8755-8f3abd9e2983", - "resource": { - "resourceType": "Observation", - "id": "b589fc38-f806-41e6-8755-8f3abd9e2983", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 92.859, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", - "resource": { - "resourceType": "Observation", - "id": "a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 32.219, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc612883-1917-4b60-9840-e7181c4a802f", - "resource": { - "resourceType": "Observation", - "id": "dc612883-1917-4b60-9840-e7181c4a802f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 35.773, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5f505a06-8b12-495a-83fd-7c666e256bad", - "resource": { - "resourceType": "Observation", - "id": "5f505a06-8b12-495a-83fd-7c666e256bad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 40.837, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bc93cd10-828d-4885-a907-66301769098b", - "resource": { - "resourceType": "Observation", - "id": "bc93cd10-828d-4885-a907-66301769098b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 312.35, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:33000cb4-08aa-4857-9870-cc14c045d611", - "resource": { - "resourceType": "Observation", - "id": "33000cb4-08aa-4857-9870-cc14c045d611", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 467.32, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7779ea12-b5d6-4ef5-bc39-34380458db93", - "resource": { - "resourceType": "Observation", - "id": "7779ea12-b5d6-4ef5-bc39-34380458db93", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueQuantity": { - "value": 9.9534, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b3938970-0a8f-46d9-96b5-46c96cf7d841", - "resource": { - "resourceType": "Observation", - "id": "b3938970-0a8f-46d9-96b5-46c96cf7d841", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f13170b9-4748-4c96-bd9f-f571aacfbe2e", - "resource": { - "resourceType": "Immunization", - "id": "f13170b9-4748-4c96-bd9f-f571aacfbe2e", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "115", - "display": "Tdap" - } - ], - "text": "Tdap" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "date": "2019-05-16T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:cd45550e-e9b8-4c57-9bc4-ee7c99462b6f", - "resource": { - "resourceType": "Immunization", - "id": "cd45550e-e9b8-4c57-9bc4-ee7c99462b6f", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "date": "2019-05-16T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:62808d09-f11f-4fa2-a367-b0c95444fc86", - "resource": { - "resourceType": "Immunization", - "id": "62808d09-f11f-4fa2-a367-b0c95444fc86", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "62", - "display": "HPV, quadrivalent" - } - ], - "text": "HPV, quadrivalent" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "date": "2019-05-16T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f4c97d17-a9ca-4464-aa6f-33c75ee5acbb", - "resource": { - "resourceType": "Immunization", - "id": "f4c97d17-a9ca-4464-aa6f-33c75ee5acbb", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "encounter": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "date": "2019-05-16T23:41:28-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:aa02cc24-70a4-4ee7-b7f0-6fa3f264e1a0", - "resource": { - "resourceType": "DiagnosticReport", - "id": "aa02cc24-70a4-4ee7-b7f0-6fa3f264e1a0", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "context": { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - }, - "effectiveDateTime": "2019-05-16T23:41:28-07:00", - "issued": "2019-05-16T23:41:28.175-07:00", - "result": [ - { - "reference": "urn:uuid:9cfb44ff-4922-493b-9e66-fa1b6f269161", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:0033e9f6-3b36-4a23-8839-d727c750fa93", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:3c46285f-0b7b-4bc6-be32-a2f9db82cca2", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:6e7a378e-93e4-45a0-b2f2-af1ef65b3d2b", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:b589fc38-f806-41e6-8755-8f3abd9e2983", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:a6ffc983-e604-4bcd-baf0-ff45ed8ae7d9", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:dc612883-1917-4b60-9840-e7181c4a802f", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:5f505a06-8b12-495a-83fd-7c666e256bad", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:bc93cd10-828d-4885-a907-66301769098b", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:33000cb4-08aa-4857-9870-cc14c045d611", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:7779ea12-b5d6-4ef5-bc39-34380458db93", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:7fbe2cfb-5891-41d9-bbd0-575b81fb2291", - "resource": { - "resourceType": "Claim", - "id": "7fbe2cfb-5891-41d9-bbd0-575b81fb2291", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "start": "2019-05-16T23:41:28-07:00", - "end": "2019-05-16T23:56:28-07:00" - }, - "organization": { - "reference": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f13170b9-4748-4c96-bd9f-f571aacfbe2e" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:cd45550e-e9b8-4c57-9bc4-ee7c99462b6f" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:62808d09-f11f-4fa2-a367-b0c95444fc86" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:f4c97d17-a9ca-4464-aa6f-33c75ee5acbb" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:eceec331-ba5c-4b19-b3e1-51c1faabf52e" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8642d6ad-3f86-4afa-a804-2befa7a5deb3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8642d6ad-3f86-4afa-a804-2befa7a5deb3", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Cigna Health" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7fbe2cfb-5891-41d9-bbd0-575b81fb2291" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:74779846-85a4-4b26-9da4-414a5fec1aed" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-05-16T23:56:28-07:00", - "end": "2020-05-16T23:56:28-07:00" - }, - "provider": { - "identifier": { - "value": "3dde31b6-dbc2-394b-b2af-a3167d1f6841" - } - }, - "organization": { - "identifier": { - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 449.66400000000004, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json deleted file mode 100644 index ae6d3327430c..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/Shavonne800_Hilll811_3024090f-fe14-40a9-8fae-79952d3c95ce.json +++ /dev/null @@ -1,32061 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce", - "resource": { - "resourceType": "Patient", - "id": "3024090f-fe14-40a9-8fae-79952d3c95ce", - "text": { - "status": "generated", - "div": "
    Generated by Synthea.Version identifier: v2.5.0-340-gabc9fc4a\n . Person seed: 8703012480265391650 Population seed: 1586298193823
    " - }, - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-race", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2106-3", - "display": "White" - } - }, - { - "url": "text", - "valueString": "White" - } - ] - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-ethnicity", - "extension": [ - { - "url": "ombCategory", - "valueCoding": { - "system": "urn:oid:2.16.840.1.113883.6.238", - "code": "2186-5", - "display": "Not Hispanic or Latino" - } - }, - { - "url": "text", - "valueString": "Not Hispanic or Latino" - } - ] - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/patient-mothersMaidenName", - "valueString": "Mary779 Green467" - }, - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-birthsex", - "valueCode": "F" - }, - { - "url": "http://hl7.org/fhir/StructureDefinition/birthPlace", - "valueAddress": { - "city": "Danvers", - "state": "Massachusetts", - "country": "US" - } - }, - { - "url": "http://synthetichealth.github.io/synthea/disability-adjusted-life-years", - "valueDecimal": 0.0 - }, - { - "url": "http://synthetichealth.github.io/synthea/quality-adjusted-life-years", - "valueDecimal": 25.0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "MR", - "display": "Medical Record Number" - } - ], - "text": "Medical Record Number" - }, - "system": "http://hospital.smarthealthit.org", - "value": "3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/identifier-type", - "code": "SB", - "display": "Social Security Number" - } - ], - "text": "Social Security Number" - }, - "system": "http://hl7.org/fhir/sid/us-ssn", - "value": "999-50-6254" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "DL", - "display": "Driver's License" - } - ], - "text": "Driver's License" - }, - "system": "urn:oid:2.16.840.1.113883.4.3.25", - "value": "S99931895" - }, - { - "type": { - "coding": [ - { - "system": "http://hl7.org/fhir/v2/0203", - "code": "PPN", - "display": "Passport Number" - } - ], - "text": "Passport Number" - }, - "system": "http://standardhealthrecord.org/fhir/StructureDefinition/passportNumber", - "value": "X49969037X" - } - ], - "name": [ - { - "use": "official", - "family": "Hilll811", - "given": [ - "Shavonne800" - ], - "prefix": [ - "Ms." - ] - } - ], - "telecom": [ - { - "system": "phone", - "value": "555-145-6552", - "use": "home" - } - ], - "gender": "female", - "birthDate": "1994-09-06", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.173297866780715 - }, - { - "url": "longitude", - "valueDecimal": -72.21882892611714 - } - ] - } - ], - "line": [ - "253 Hackett Meadow Unit 91" - ], - "city": "Warren", - "state": "Massachusetts", - "country": "US" - } - ], - "maritalStatus": { - "coding": [ - { - "system": "http://hl7.org/fhir/v3/MaritalStatus", - "code": "S", - "display": "Never Married" - } - ], - "text": "Never Married" - }, - "multipleBirthInteger": 1, - "communication": [ - { - "language": { - "coding": [ - { - "system": "urn:ietf:bcp:47", - "code": "en-US", - "display": "English" - } - ], - "text": "English" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Patient" - } - }, - { - "fullUrl": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03", - "resource": { - "resourceType": "Organization", - "id": "5844ad77-f653-3c2b-b7dd-e97576ab3b03", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "BAYSTATE WING HOSPITAL AND MEDICAL CENTERS", - "telecom": [ - { - "system": "phone", - "value": "4132837651" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.187794 - }, - { - "url": "longitude", - "valueDecimal": -72.30846899999997 - } - ] - } - ], - "line": [ - "40 WRIGHT STREET" - ], - "city": "PALMER", - "state": "MA", - "postalCode": "01069", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9", - "resource": { - "resourceType": "Practitioner", - "id": "42f9935b-696e-37c9-8261-2a8c72d02dc9", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "130" - } - ], - "active": true, - "name": [ - { - "family": "Abernathy524", - "given": [ - "Kirby843" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "40 WRIGHT STREET" - ], - "city": "PALMER", - "state": "MA", - "postalCode": "01069", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5", - "resource": { - "resourceType": "Encounter", - "id": "1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2009-08-04T16:24:16-07:00", - "end": "2014-11-11T15:24:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:ee4121b9-5f8e-44d8-a33c-76b97b20fa75", - "resource": { - "resourceType": "Claim", - "id": "ee4121b9-5f8e-44d8-a33c-76b97b20fa75", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2009-08-04T16:24:16-07:00", - "end": "2014-11-11T15:24:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:26cbb6a0-9952-405f-9965-d638c86b19c9", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "26cbb6a0-9952-405f-9965-d638c86b19c9", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ee4121b9-5f8e-44d8-a33c-76b97b20fa75" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2015-11-11T15:24:16-08:00" - }, - "created": "2014-11-11T15:24:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:ee4121b9-5f8e-44d8-a33c-76b97b20fa75" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2009-08-04T16:24:16-07:00", - "end": "2014-11-11T15:24:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1ea7fbd2-cecb-49fc-b9b8-ec9739c5b0a5" - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a", - "resource": { - "resourceType": "Encounter", - "id": "1bb4a6d8-4e87-448d-8765-0656cc53887a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2010-08-23T16:24:16-07:00", - "end": "2010-08-23T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a5cef09d-5e28-44fc-8c85-64ef2058260f", - "resource": { - "resourceType": "MedicationRequest", - "id": "a5cef09d-5e28-44fc-8c85-64ef2058260f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "748856", - "display": "Yaz 28 Day Pack" - } - ], - "text": "Yaz 28 Day Pack" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" - }, - "authoredOn": "2010-08-23T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:db886643-1469-4967-b699-399c05307706", - "resource": { - "resourceType": "Claim", - "id": "db886643-1469-4967-b699-399c05307706", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2010-08-23T16:24:16-07:00", - "end": "2010-08-23T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:a5cef09d-5e28-44fc-8c85-64ef2058260f" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" - } - ] - } - ], - "total": { - "value": 33.23, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b01e9050-0b65-4d8b-9695-76c3d2c055ee", - "resource": { - "resourceType": "Claim", - "id": "b01e9050-0b65-4d8b-9695-76c3d2c055ee", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2010-08-23T16:24:16-07:00", - "end": "2010-08-23T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3453fd2d-cd9c-462c-81a6-5e1062c46348", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3453fd2d-cd9c-462c-81a6-5e1062c46348", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b01e9050-0b65-4d8b-9695-76c3d2c055ee" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2010-08-23T16:39:16-07:00", - "end": "2011-08-23T16:39:16-07:00" - }, - "created": "2010-08-23T16:39:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:b01e9050-0b65-4d8b-9695-76c3d2c055ee" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2010-08-23T16:24:16-07:00", - "end": "2010-08-23T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1bb4a6d8-4e87-448d-8765-0656cc53887a" - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4", - "resource": { - "resourceType": "Organization", - "id": "883860d6-0862-3a47-a18f-7256563c53f4", - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "HARRINGTON PHYSICIAN SERVICES INC", - "telecom": [ - { - "system": "phone", - "value": "413-245-0966" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.128176 - }, - { - "url": "longitude", - "valueDecimal": -72.205352 - } - ] - } - ], - "line": [ - "255 E OLD STURBRIDGE RD" - ], - "city": "BRIMFIELD", - "state": "MA", - "postalCode": "01010-9647", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915", - "resource": { - "resourceType": "Practitioner", - "id": "7212c355-6c66-3c09-b527-bb8d33b0a915", - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "85760" - } - ], - "active": true, - "name": [ - { - "family": "Kihn564", - "given": [ - "Hayden835" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "255 E OLD STURBRIDGE RD" - ], - "city": "BRIMFIELD", - "state": "MA", - "postalCode": "01010-9647", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:34954861-c50f-48d1-9751-45829b517826", - "resource": { - "resourceType": "Encounter", - "id": "34954861-c50f-48d1-9751-45829b517826", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2010-10-19T16:24:16-07:00", - "end": "2010-10-19T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b147e4ca-490a-4a11-91a8-d3df387ff13a", - "resource": { - "resourceType": "Observation", - "id": "b147e4ca-490a-4a11-91a8-d3df387ff13a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 159.80, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:29399f80-6863-4915-9942-bfff6fb83802", - "resource": { - "resourceType": "Observation", - "id": "29399f80-6863-4915-9942-bfff6fb83802", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32953059-5bb3-4e1e-913a-20f36e41d1e4", - "resource": { - "resourceType": "Observation", - "id": "32953059-5bb3-4e1e-913a-20f36e41d1e4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 47.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:85f2dcf5-0ce9-4f91-8056-340e74e7895f", - "resource": { - "resourceType": "Observation", - "id": "85f2dcf5-0ce9-4f91-8056-340e74e7895f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 18.590, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:446d7ae7-4543-419a-ace1-91ec13043ace", - "resource": { - "resourceType": "Observation", - "id": "446d7ae7-4543-419a-ace1-91ec13043ace", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 23.352, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:66f0f611-87c6-4004-bad8-93d20c19a174", - "resource": { - "resourceType": "Observation", - "id": "66f0f611-87c6-4004-bad8-93d20c19a174", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 120, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:43440109-3988-466b-82df-9af305f69234", - "resource": { - "resourceType": "Observation", - "id": "43440109-3988-466b-82df-9af305f69234", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 62, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:67bf611b-32b7-4007-86dc-670418f13b38", - "resource": { - "resourceType": "Observation", - "id": "67bf611b-32b7-4007-86dc-670418f13b38", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:78401336-35e4-47c5-be88-10d360390866", - "resource": { - "resourceType": "Observation", - "id": "78401336-35e4-47c5-be88-10d360390866", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 8.5499, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:472a834e-bd60-42ef-83b3-0aac2d7596fa", - "resource": { - "resourceType": "Observation", - "id": "472a834e-bd60-42ef-83b3-0aac2d7596fa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 4.4570, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", - "resource": { - "resourceType": "Observation", - "id": "25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 13.488, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cb528262-4f2b-4535-af32-e1cd91ede53b", - "resource": { - "resourceType": "Observation", - "id": "cb528262-4f2b-4535-af32-e1cd91ede53b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 44.643, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32a699de-77e5-4e01-85e6-8b1c5f2b04c2", - "resource": { - "resourceType": "Observation", - "id": "32a699de-77e5-4e01-85e6-8b1c5f2b04c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 89.460, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:511863d5-34ab-41f3-a5aa-5f15bab65932", - "resource": { - "resourceType": "Observation", - "id": "511863d5-34ab-41f3-a5aa-5f15bab65932", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 32.373, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8481b85d-8ee6-40c3-891f-289b3e682fad", - "resource": { - "resourceType": "Observation", - "id": "8481b85d-8ee6-40c3-891f-289b3e682fad", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 35.068, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:be946655-8e39-4e40-9f21-a2628b9438a9", - "resource": { - "resourceType": "Observation", - "id": "be946655-8e39-4e40-9f21-a2628b9438a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 39.197, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:efb11be8-0ea1-40ac-84ce-443106143724", - "resource": { - "resourceType": "Observation", - "id": "efb11be8-0ea1-40ac-84ce-443106143724", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 331.86, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:2646f8a3-8c1d-4908-98d6-9540d57246b2", - "resource": { - "resourceType": "Observation", - "id": "2646f8a3-8c1d-4908-98d6-9540d57246b2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 256.08, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:12ccbcb3-76c0-4d8d-966b-0bede82b17fe", - "resource": { - "resourceType": "Observation", - "id": "12ccbcb3-76c0-4d8d-966b-0bede82b17fe", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueQuantity": { - "value": 10.098, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0cec3548-cd55-4b06-884f-47e45297afb1", - "resource": { - "resourceType": "Observation", - "id": "0cec3548-cd55-4b06-884f-47e45297afb1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c89ea1ae-21e5-4af9-8c74-770c20bca76b", - "resource": { - "resourceType": "Immunization", - "id": "c89ea1ae-21e5-4af9-8c74-770c20bca76b", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "date": "2010-10-19T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e8092f33-962e-425d-82a4-5c88cf5621cd", - "resource": { - "resourceType": "Immunization", - "id": "e8092f33-962e-425d-82a4-5c88cf5621cd", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "date": "2010-10-19T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:4fc2be12-efb2-40b0-9c96-7505833a0f4b", - "resource": { - "resourceType": "DiagnosticReport", - "id": "4fc2be12-efb2-40b0-9c96-7505833a0f4b", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - }, - "effectiveDateTime": "2010-10-19T16:24:16-07:00", - "issued": "2010-10-19T16:24:16.644-07:00", - "result": [ - { - "reference": "urn:uuid:78401336-35e4-47c5-be88-10d360390866", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:472a834e-bd60-42ef-83b3-0aac2d7596fa", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:25112a0b-d9c9-4dc8-94c9-15e1547d2cbd", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:cb528262-4f2b-4535-af32-e1cd91ede53b", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:32a699de-77e5-4e01-85e6-8b1c5f2b04c2", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:511863d5-34ab-41f3-a5aa-5f15bab65932", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:8481b85d-8ee6-40c3-891f-289b3e682fad", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:be946655-8e39-4e40-9f21-a2628b9438a9", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:efb11be8-0ea1-40ac-84ce-443106143724", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:2646f8a3-8c1d-4908-98d6-9540d57246b2", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:12ccbcb3-76c0-4d8d-966b-0bede82b17fe", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:9b8f71ed-15d7-419b-81cd-71c2e0643795", - "resource": { - "resourceType": "Claim", - "id": "9b8f71ed-15d7-419b-81cd-71c2e0643795", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2010-10-19T16:24:16-07:00", - "end": "2010-10-19T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:c89ea1ae-21e5-4af9-8c74-770c20bca76b" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e8092f33-962e-425d-82a4-5c88cf5621cd" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:34954861-c50f-48d1-9751-45829b517826" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:f1244e38-710f-44fd-b2fa-383f2331fc28", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "f1244e38-710f-44fd-b2fa-383f2331fc28", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9b8f71ed-15d7-419b-81cd-71c2e0643795" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2010-10-19T16:39:16-07:00", - "end": "2011-10-19T16:39:16-07:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 224.83200000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a", - "resource": { - "resourceType": "Encounter", - "id": "4c5f6878-6daa-483c-a0eb-1f36da49417a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2011-08-18T16:24:16-07:00", - "end": "2011-08-18T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e680310c-0089-4f8b-bb75-0e53f6289636", - "resource": { - "resourceType": "MedicationRequest", - "id": "e680310c-0089-4f8b-bb75-0e53f6289636", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "978950", - "display": "Natazia 28 Day Pack" - } - ], - "text": "Natazia 28 Day Pack" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" - }, - "authoredOn": "2011-08-18T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:6e400c21-7cb8-41be-b543-cef90ca4c662", - "resource": { - "resourceType": "Claim", - "id": "6e400c21-7cb8-41be-b543-cef90ca4c662", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2011-08-18T16:24:16-07:00", - "end": "2011-08-18T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:e680310c-0089-4f8b-bb75-0e53f6289636" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" - } - ] - } - ], - "total": { - "value": 46.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d96cab55-0542-418e-b436-7ca1718f8c5b", - "resource": { - "resourceType": "Claim", - "id": "d96cab55-0542-418e-b436-7ca1718f8c5b", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2011-08-18T16:24:16-07:00", - "end": "2011-08-18T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6c15bc73-870d-4ce6-80c1-bd60dc1bbb78", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6c15bc73-870d-4ce6-80c1-bd60dc1bbb78", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d96cab55-0542-418e-b436-7ca1718f8c5b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2011-08-18T16:39:16-07:00", - "end": "2012-08-18T16:39:16-07:00" - }, - "created": "2011-08-18T16:39:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:d96cab55-0542-418e-b436-7ca1718f8c5b" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2011-08-18T16:24:16-07:00", - "end": "2011-08-18T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:4c5f6878-6daa-483c-a0eb-1f36da49417a" - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022", - "resource": { - "resourceType": "Encounter", - "id": "b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "410620009", - "display": "Well child visit (procedure)" - } - ], - "text": "Well child visit (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2011-10-25T16:24:16-07:00", - "end": "2011-10-25T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b266256f-9b16-48c5-8730-35e8f34cf460", - "resource": { - "resourceType": "Observation", - "id": "b266256f-9b16-48c5-8730-35e8f34cf460", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 160.10, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a54bb9ba-7de2-4b82-b271-8b2eef095de8", - "resource": { - "resourceType": "Observation", - "id": "a54bb9ba-7de2-4b82-b271-8b2eef095de8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b0e356ce-67e9-4dc1-a228-1ae10dd1a1a5", - "resource": { - "resourceType": "Observation", - "id": "b0e356ce-67e9-4dc1-a228-1ae10dd1a1a5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 46.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7cc12c6a-ae44-4626-b1b6-aed45de8ca5b", - "resource": { - "resourceType": "Observation", - "id": "7cc12c6a-ae44-4626-b1b6-aed45de8ca5b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 18.140, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0b89178a-8b44-4bf6-afe4-6462b9c790a9", - "resource": { - "resourceType": "Observation", - "id": "0b89178a-8b44-4bf6-afe4-6462b9c790a9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 12.593, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:244a1173-04bb-4cf9-9390-4cdf653ab566", - "resource": { - "resourceType": "Observation", - "id": "244a1173-04bb-4cf9-9390-4cdf653ab566", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 83, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 126, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:22277078-cea0-4664-be13-df195a78fc21", - "resource": { - "resourceType": "Observation", - "id": "22277078-cea0-4664-be13-df195a78fc21", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 85, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a3a897e-b6d2-4fc3-911e-42480e3a629b", - "resource": { - "resourceType": "Observation", - "id": "5a3a897e-b6d2-4fc3-911e-42480e3a629b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a615ac44-517a-46c1-8b2d-32606224a860", - "resource": { - "resourceType": "Observation", - "id": "a615ac44-517a-46c1-8b2d-32606224a860", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "effectiveDateTime": "2011-10-25T16:24:16-07:00", - "issued": "2011-10-25T16:24:16.644-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc890bd2-3e42-4fac-9cb5-631967c34989", - "resource": { - "resourceType": "Immunization", - "id": "fc890bd2-3e42-4fac-9cb5-631967c34989", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - }, - "date": "2011-10-25T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:f91dc1d7-0237-42ca-b1dc-4eee732f3af3", - "resource": { - "resourceType": "Claim", - "id": "f91dc1d7-0237-42ca-b1dc-4eee732f3af3", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2011-10-25T16:24:16-07:00", - "end": "2011-10-25T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:fc890bd2-3e42-4fac-9cb5-631967c34989" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b2ed80c2-ee7c-4f1b-a3a4-7adeffdd9022" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:627db0a1-4778-4cc9-a0d5-949bce0e1471", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "627db0a1-4778-4cc9-a0d5-949bce0e1471", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f91dc1d7-0237-42ca-b1dc-4eee732f3af3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2011-10-25T16:39:16-07:00", - "end": "2012-10-25T16:39:16-07:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305", - "resource": { - "resourceType": "Encounter", - "id": "13af4cf2-ed03-4441-b836-db5066581305", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2012-10-30T16:24:16-07:00", - "end": "2012-10-30T16:54:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701", - "resource": { - "resourceType": "Condition", - "id": "42372dca-1642-40a5-acfa-8ecddbf32701", - "clinicalStatus": "active", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "59621000", - "display": "Hypertension" - } - ], - "text": "Hypertension" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "onsetDateTime": "2012-10-30T16:24:16-07:00", - "assertedDate": "2012-10-30T16:24:16-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:706460c4-3803-417a-8a50-df8c417fee87", - "resource": { - "resourceType": "Observation", - "id": "706460c4-3803-417a-8a50-df8c417fee87", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 160.30, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5748203a-0471-4992-8a6e-9a65e21897b7", - "resource": { - "resourceType": "Observation", - "id": "5748203a-0471-4992-8a6e-9a65e21897b7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8618ef2a-0916-41ac-917e-cb16989d6f7b", - "resource": { - "resourceType": "Observation", - "id": "8618ef2a-0916-41ac-917e-cb16989d6f7b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 44.900, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:68e86fb9-556c-4a89-8263-faa03fe0d921", - "resource": { - "resourceType": "Observation", - "id": "68e86fb9-556c-4a89-8263-faa03fe0d921", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 17.460, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:50b4c213-652c-4f14-a77a-097fffe2f2eb", - "resource": { - "resourceType": "Observation", - "id": "50b4c213-652c-4f14-a77a-097fffe2f2eb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 4.3434, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:87c89a21-dc08-4885-8f47-77ac9c5990fd", - "resource": { - "resourceType": "Observation", - "id": "87c89a21-dc08-4885-8f47-77ac9c5990fd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 99, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 173, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11b3b668-c3ac-4cf3-98dd-c5d54eea098e", - "resource": { - "resourceType": "Observation", - "id": "11b3b668-c3ac-4cf3-98dd-c5d54eea098e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 74, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5bed8478-2ac4-4bd4-94ae-cd2f07b71943", - "resource": { - "resourceType": "Observation", - "id": "5bed8478-2ac4-4bd4-94ae-cd2f07b71943", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b07409ec-7296-4831-838b-cbad2533a94d", - "resource": { - "resourceType": "Observation", - "id": "b07409ec-7296-4831-838b-cbad2533a94d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "effectiveDateTime": "2012-10-30T16:24:16-07:00", - "issued": "2012-10-30T16:24:16.644-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c4b229f-4426-4699-8239-f0246850fc51", - "resource": { - "resourceType": "Procedure", - "id": "7c4b229f-4426-4699-8239-f0246850fc51", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "performedPeriod": { - "start": "2012-10-30T16:24:16-07:00", - "end": "2012-10-30T16:39:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3ab5b202-9a2e-48d3-9bb9-3b7304217e4d", - "resource": { - "resourceType": "MedicationRequest", - "id": "3ab5b202-9a2e-48d3-9bb9-3b7304217e4d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "310798", - "display": "Hydrochlorothiazide 25 MG Oral Tablet" - } - ], - "text": "Hydrochlorothiazide 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "authoredOn": "2012-10-30T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:404c81af-98fb-44fe-878b-974fe397a1f6", - "resource": { - "resourceType": "Claim", - "id": "404c81af-98fb-44fe-878b-974fe397a1f6", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2012-10-30T16:24:16-07:00", - "end": "2012-10-30T16:54:16-07:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:3ab5b202-9a2e-48d3-9bb9-3b7304217e4d" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:faa7b756-3a02-48c5-bed2-4dbc5b952409", - "resource": { - "resourceType": "Immunization", - "id": "faa7b756-3a02-48c5-bed2-4dbc5b952409", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "date": "2012-10-30T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:df7f7291-36bd-4eda-92c8-43b22eccf514", - "resource": { - "resourceType": "CarePlan", - "id": "df7f7291-36bd-4eda-92c8-43b22eccf514", - "status": "active", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "443402002", - "display": "Lifestyle education regarding hypertension" - } - ], - "text": "Lifestyle education regarding hypertension" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - }, - "period": { - "start": "2012-10-30T16:24:16-07:00" - }, - "addresses": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "386463000", - "display": "Prescribed activity/exercise education" - } - ], - "text": "Prescribed activity/exercise education" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "413473000", - "display": "Counseling about alcohol consumption" - } - ], - "text": "Counseling about alcohol consumption" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "1151000175103", - "display": "Dietary approaches to stop hypertension diet" - } - ], - "text": "Dietary approaches to stop hypertension diet" - }, - "status": "in-progress" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "225323000", - "display": "Smoking cessation education" - } - ], - "text": "Smoking cessation education" - }, - "status": "in-progress" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:2ddfac78-0bc1-4978-9286-ee41d0c85454", - "resource": { - "resourceType": "Claim", - "id": "2ddfac78-0bc1-4978-9286-ee41d0c85454", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2012-10-30T16:24:16-07:00", - "end": "2012-10-30T16:54:16-07:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:faa7b756-3a02-48c5-bed2-4dbc5b952409" - } - } - ], - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7c4b229f-4426-4699-8239-f0246850fc51" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:13af4cf2-ed03-4441-b836-db5066581305" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 421.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dc1460f2-e563-4161-b451-f88a9981cb05", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "dc1460f2-e563-4161-b451-f88a9981cb05", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "2ddfac78-0bc1-4978-9286-ee41d0c85454" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-10-30T16:54:16-07:00", - "end": "2013-10-30T16:54:16-07:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 84.34800000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 337.39200000000005, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 421.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 421.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 449.80800000000005, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de", - "resource": { - "resourceType": "Encounter", - "id": "2fae00ba-3d25-492e-908c-38d0475074de", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2012-11-29T15:24:16-08:00", - "end": "2012-11-29T15:39:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bdda12d8-b3c4-4415-9fd8-f467170240f5", - "resource": { - "resourceType": "Observation", - "id": "bdda12d8-b3c4-4415-9fd8-f467170240f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" - }, - "effectiveDateTime": "2012-11-29T15:24:16-08:00", - "issued": "2012-11-29T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 192, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:483cbb0e-c06e-4a0b-87b3-e9abadab17e2", - "resource": { - "resourceType": "MedicationRequest", - "id": "483cbb0e-c06e-4a0b-87b3-e9abadab17e2", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" - }, - "authoredOn": "2012-11-29T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0cd15c2c-68d9-4d64-b00a-40ce61e6bf16", - "resource": { - "resourceType": "Claim", - "id": "0cd15c2c-68d9-4d64-b00a-40ce61e6bf16", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2012-11-29T15:24:16-08:00", - "end": "2012-11-29T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:483cbb0e-c06e-4a0b-87b3-e9abadab17e2" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:917196aa-7ef3-478a-959b-b1afec00a7e0", - "resource": { - "resourceType": "Claim", - "id": "917196aa-7ef3-478a-959b-b1afec00a7e0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2012-11-29T15:24:16-08:00", - "end": "2012-11-29T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:2fae00ba-3d25-492e-908c-38d0475074de" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e3042db7-c3c7-42a3-adec-ff4ed4b57e0f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e3042db7-c3c7-42a3-adec-ff4ed4b57e0f", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "917196aa-7ef3-478a-959b-b1afec00a7e0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2012-11-29T15:39:16-08:00", - "end": "2013-11-29T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03", - "resource": { - "resourceType": "Encounter", - "id": "4f04379e-6a5e-4e7e-9898-c3391412cd03", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "390906007", - "display": "Hypertension follow-up encounter" - } - ], - "text": "Hypertension follow-up encounter" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2013-01-28T15:24:16-08:00", - "end": "2013-01-28T15:39:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:118f1665-cf5f-46b6-9464-c3b04239830e", - "resource": { - "resourceType": "Observation", - "id": "118f1665-cf5f-46b6-9464-c3b04239830e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03" - }, - "effectiveDateTime": "2013-01-28T15:24:16-08:00", - "issued": "2013-01-28T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5a7a60f1-45a9-485a-905c-73f129772ede", - "resource": { - "resourceType": "Claim", - "id": "5a7a60f1-45a9-485a-905c-73f129772ede", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2013-01-28T15:24:16-08:00", - "end": "2013-01-28T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:4f04379e-6a5e-4e7e-9898-c3391412cd03" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a43f503c-90d6-45f8-b350-e5f40fa4c470", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a43f503c-90d6-45f8-b350-e5f40fa4c470", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5a7a60f1-45a9-485a-905c-73f129772ede" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-01-28T15:39:16-08:00", - "end": "2014-01-28T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0", - "resource": { - "resourceType": "Encounter", - "id": "d2c47f97-7465-4a1f-9a54-d2641f2e55b0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2013-11-05T15:24:16-08:00", - "end": "2013-11-05T15:39:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:fdb5362f-f3f9-4f15-9297-c70b7c7f655e", - "resource": { - "resourceType": "Observation", - "id": "fdb5362f-f3f9-4f15-9297-c70b7c7f655e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.40, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:014700a0-bd9a-458d-8276-fe76e28aab16", - "resource": { - "resourceType": "Observation", - "id": "014700a0-bd9a-458d-8276-fe76e28aab16", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b6981ec3-4498-4142-b63c-32960f654629", - "resource": { - "resourceType": "Observation", - "id": "b6981ec3-4498-4142-b63c-32960f654629", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 46.300, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c7403f81-5bc3-434c-bec3-08b6e13b3943", - "resource": { - "resourceType": "Observation", - "id": "c7403f81-5bc3-434c-bec3-08b6e13b3943", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 17.990, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d4996166-9094-4327-b4ff-ebdfd9bec972", - "resource": { - "resourceType": "Observation", - "id": "d4996166-9094-4327-b4ff-ebdfd9bec972", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 6.4173, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fda835e4-a508-42bb-8a8c-8f23e5e746b0", - "resource": { - "resourceType": "Observation", - "id": "fda835e4-a508-42bb-8a8c-8f23e5e746b0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 74, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 108, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f883ddde-e985-4fc4-b6d0-92e14a2d7782", - "resource": { - "resourceType": "Observation", - "id": "f883ddde-e985-4fc4-b6d0-92e14a2d7782", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 75, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e17d2564-008a-45ec-8876-04e292e2a4df", - "resource": { - "resourceType": "Observation", - "id": "e17d2564-008a-45ec-8876-04e292e2a4df", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4cdfb7e2-257d-40bd-9c4c-d065cf38367b", - "resource": { - "resourceType": "Observation", - "id": "4cdfb7e2-257d-40bd-9c4c-d065cf38367b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "effectiveDateTime": "2013-11-05T15:24:16-08:00", - "issued": "2013-11-05T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ef780297-3f41-4bca-be9b-1895b9cb2e9d", - "resource": { - "resourceType": "MedicationRequest", - "id": "ef780297-3f41-4bca-be9b-1895b9cb2e9d", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "authoredOn": "2013-11-05T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:d35d1d98-f587-4ead-8605-943efe14675d", - "resource": { - "resourceType": "Claim", - "id": "d35d1d98-f587-4ead-8605-943efe14675d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2013-11-05T15:24:16-08:00", - "end": "2013-11-05T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:ef780297-3f41-4bca-be9b-1895b9cb2e9d" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8480205d-e9b2-4df3-a9a6-b3d86e02f187", - "resource": { - "resourceType": "Immunization", - "id": "8480205d-e9b2-4df3-a9a6-b3d86e02f187", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - }, - "date": "2013-11-05T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877", - "resource": { - "resourceType": "Claim", - "id": "ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2013-11-05T15:24:16-08:00", - "end": "2013-11-05T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:8480205d-e9b2-4df3-a9a6-b3d86e02f187" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:d2c47f97-7465-4a1f-9a54-d2641f2e55b0" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d480df78-9a3d-4ae4-9079-32c71f3d22a3", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d480df78-9a3d-4ae4-9079-32c71f3d22a3", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ca98ec0e-e7ef-4444-bcaa-c3a4d6cda877" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2013-11-05T15:39:16-08:00", - "end": "2014-11-05T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517", - "resource": { - "resourceType": "Encounter", - "id": "e67b8297-a732-4b85-9489-4837ff0fd517", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2014-03-18T16:24:16-07:00", - "end": "2014-03-18T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e4d261dc-f1fe-41aa-9ae0-852c85171753", - "resource": { - "resourceType": "MedicationRequest", - "id": "e4d261dc-f1fe-41aa-9ae0-852c85171753", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" - }, - "authoredOn": "2014-03-18T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:4c95154c-38d9-45be-9a4f-ad8789f56784", - "resource": { - "resourceType": "Claim", - "id": "4c95154c-38d9-45be-9a4f-ad8789f56784", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-18T16:24:16-07:00", - "end": "2014-03-18T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:e4d261dc-f1fe-41aa-9ae0-852c85171753" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:70d584d4-67c1-4761-8604-ee4308d7fa60", - "resource": { - "resourceType": "Immunization", - "id": "70d584d4-67c1-4761-8604-ee4308d7fa60", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" - }, - "date": "2014-03-18T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:887c3e1a-a14d-4d4b-81e9-49ab8aa67d46", - "resource": { - "resourceType": "Claim", - "id": "887c3e1a-a14d-4d4b-81e9-49ab8aa67d46", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-18T16:24:16-07:00", - "end": "2014-03-18T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:70d584d4-67c1-4761-8604-ee4308d7fa60" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:83341480-28ee-4f70-a1e1-c8c25e339fca", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "83341480-28ee-4f70-a1e1-c8c25e339fca", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "887c3e1a-a14d-4d4b-81e9-49ab8aa67d46" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-18T16:39:16-07:00", - "end": "2015-03-18T16:39:16-07:00" - }, - "created": "2014-03-18T16:39:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:887c3e1a-a14d-4d4b-81e9-49ab8aa67d46" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-03-18T16:24:16-07:00", - "end": "2014-03-18T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e67b8297-a732-4b85-9489-4837ff0fd517" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-03-18T16:24:16-07:00", - "end": "2014-03-18T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667", - "resource": { - "resourceType": "Encounter", - "id": "ee580418-5e8c-421b-8671-952e139f4667", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-03-13T17:00:16-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", - "resource": { - "resourceType": "Condition", - "id": "e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10509002", - "display": "Acute bronchitis (disorder)" - } - ], - "text": "Acute bronchitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "onsetDateTime": "2014-03-13T16:24:16-07:00", - "abatementDateTime": "2014-03-27T16:24:16-07:00", - "assertedDate": "2014-03-13T16:24:16-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:59fa98f4-ff20-4f7d-b532-fe647f635944", - "resource": { - "resourceType": "Observation", - "id": "59fa98f4-ff20-4f7d-b532-fe647f635944", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7917c548-f7c1-4b08-9cf7-7d23a63684ee", - "resource": { - "resourceType": "Observation", - "id": "7917c548-f7c1-4b08-9cf7-7d23a63684ee", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3b6f80c1-ad8f-4af7-9f84-65f56e096b9e", - "resource": { - "resourceType": "Observation", - "id": "3b6f80c1-ad8f-4af7-9f84-65f56e096b9e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 47.600, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:aeca7327-ba7a-49ad-ac01-80f694410961", - "resource": { - "resourceType": "Observation", - "id": "aeca7327-ba7a-49ad-ac01-80f694410961", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 18.480, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1107a2db-0c0c-4856-a71a-72ab01277117", - "resource": { - "resourceType": "Observation", - "id": "1107a2db-0c0c-4856-a71a-72ab01277117", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "59576-9", - "display": "Body mass index (BMI) [Percentile] Per age and gender" - } - ], - "text": "Body mass index (BMI) [Percentile] Per age and gender" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 10.111, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:80ebd06c-6695-4dac-b4ff-9682cf900ad3", - "resource": { - "resourceType": "Observation", - "id": "80ebd06c-6695-4dac-b4ff-9682cf900ad3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 76, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 113, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c7e5b77b-a63a-4cdb-90b0-512003ad8d91", - "resource": { - "resourceType": "Observation", - "id": "c7e5b77b-a63a-4cdb-90b0-512003ad8d91", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 90, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:685b71bf-1779-47d1-8925-95014ad1fbcc", - "resource": { - "resourceType": "Observation", - "id": "685b71bf-1779-47d1-8925-95014ad1fbcc", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueQuantity": { - "value": 15, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:98845d25-be1b-42e7-b8a2-bb208f0ec088", - "resource": { - "resourceType": "Observation", - "id": "98845d25-be1b-42e7-b8a2-bb208f0ec088", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "effectiveDateTime": "2014-03-18T16:24:16-07:00", - "issued": "2014-03-18T16:24:16.644-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8", - "resource": { - "resourceType": "Procedure", - "id": "b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "23426006", - "display": "Measurement of respiratory function (procedure)" - } - ], - "text": "Measurement of respiratory function (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "performedPeriod": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-03-13T16:45:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92", - "display": "Acute bronchitis (disorder)" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:301853d7-80b5-4225-9be6-9ade82f136a6", - "resource": { - "resourceType": "MedicationRequest", - "id": "301853d7-80b5-4225-9be6-9ade82f136a6", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "313782", - "display": "Acetaminophen 325 MG Oral Tablet" - } - ], - "text": "Acetaminophen 325 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "authoredOn": "2014-03-13T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:fc18db14-4893-4b6f-931c-31c6ad741c82", - "resource": { - "resourceType": "Claim", - "id": "fc18db14-4893-4b6f-931c-31c6ad741c82", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-03-13T17:00:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:301853d7-80b5-4225-9be6-9ade82f136a6" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - } - ] - } - ], - "total": { - "value": 6.53, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:73008d30-b07d-4712-a95a-118b071da6a5", - "resource": { - "resourceType": "MedicationRequest", - "id": "73008d30-b07d-4712-a95a-118b071da6a5", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "authoredOn": "2014-03-18T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:1deb2f81-ab06-417a-aa2d-a426e2be697f", - "resource": { - "resourceType": "Claim", - "id": "1deb2f81-ab06-417a-aa2d-a426e2be697f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-03-13T17:00:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:73008d30-b07d-4712-a95a-118b071da6a5" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:93fafb6a-793e-42b4-ad9e-1b9a6f2c64bb", - "resource": { - "resourceType": "CarePlan", - "id": "93fafb6a-793e-42b4-ad9e-1b9a6f2c64bb", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "53950000", - "display": "Respiratory therapy" - } - ], - "text": "Respiratory therapy" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - }, - "period": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-11-11T15:24:16-08:00" - }, - "addresses": [ - { - "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "304510005", - "display": "Recommendation to avoid exercise" - } - ], - "text": "Recommendation to avoid exercise" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "371605008", - "display": "Deep breathing and coughing exercises" - } - ], - "text": "Deep breathing and coughing exercises" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:d9537751-3ae8-42b3-8e97-1a9732e549b9", - "resource": { - "resourceType": "Claim", - "id": "d9537751-3ae8-42b3-8e97-1a9732e549b9", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-03-13T16:24:16-07:00", - "end": "2014-03-13T17:00:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b8dd44a9-5eb1-4685-b90f-d7d98cc4a0f8" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ee580418-5e8c-421b-8671-952e139f4667" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "23426006" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:95e255a3-d9e2-4451-a480-60b37e1a2d3b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "95e255a3-d9e2-4451-a480-60b37e1a2d3b", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d9537751-3ae8-42b3-8e97-1a9732e549b9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-03-13T17:00:16-07:00", - "end": "2015-03-13T17:00:16-07:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:e9b65c34-ecf6-4461-8618-2ae1d7ec2b92" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "23426006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c", - "resource": { - "resourceType": "Encounter", - "id": "26dbc751-7daf-4205-b8a6-0c74e6e5f58c", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:50:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:80d776f9-f6e7-40d5-9066-c79e84320359", - "resource": { - "resourceType": "Procedure", - "id": "80d776f9-f6e7-40d5-9066-c79e84320359", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" - }, - "performedPeriod": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:35:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3550163d-fd97-4910-bacf-490302fb4d6e", - "resource": { - "resourceType": "MedicationRequest", - "id": "3550163d-fd97-4910-bacf-490302fb4d6e", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1000126", - "display": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" - } - ], - "text": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" - }, - "authoredOn": "2014-08-02T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:5a93ecad-4365-48bc-ab30-9d313ea3599e", - "resource": { - "resourceType": "Claim", - "id": "5a93ecad-4365-48bc-ab30-9d313ea3599e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:50:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:3550163d-fd97-4910-bacf-490302fb4d6e" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" - } - ] - } - ], - "total": { - "value": 303.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5d45a5fa-8917-400b-8ee5-d6d9b6289b7d", - "resource": { - "resourceType": "Claim", - "id": "5d45a5fa-8917-400b-8ee5-d6d9b6289b7d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:50:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:80d776f9-f6e7-40d5-9066-c79e84320359" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3250.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d50d29e0-09d9-40fa-9283-0ccd3c5f3ea1", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d50d29e0-09d9-40fa-9283-0ccd3c5f3ea1", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "5d45a5fa-8917-400b-8ee5-d6d9b6289b7d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-08-02T16:50:16-07:00", - "end": "2015-08-02T16:50:16-07:00" - }, - "created": "2014-08-02T16:50:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:5d45a5fa-8917-400b-8ee5-d6d9b6289b7d" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:26dbc751-7daf-4205-b8a6-0c74e6e5f58c" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2014-08-02T16:24:16-07:00", - "end": "2014-08-02T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3250.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 650.1500000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2600.6000000000004, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3250.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3250.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2600.6000000000004, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0", - "resource": { - "resourceType": "Encounter", - "id": "a1441018-28ac-4b9d-b110-edfe104c4df0", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:39:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:52324c9a-431e-4cc2-a2f2-9b54af5780c6", - "resource": { - "resourceType": "Immunization", - "id": "52324c9a-431e-4cc2-a2f2-9b54af5780c6", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "43", - "display": "Hep B, adult" - } - ], - "text": "Hep B, adult" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0" - }, - "date": "2014-11-11T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:79403ae7-90a0-433c-8059-beac56c514b4", - "resource": { - "resourceType": "Claim", - "id": "79403ae7-90a0-433c-8059-beac56c514b4", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:52324c9a-431e-4cc2-a2f2-9b54af5780c6" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:a1441018-28ac-4b9d-b110-edfe104c4df0" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3d4254cf-74c9-446c-80c6-b281f050f23e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "3d4254cf-74c9-446c-80c6-b281f050f23e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "79403ae7-90a0-433c-8059-beac56c514b4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2014-11-11T15:39:16-08:00", - "end": "2015-11-11T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce", - "resource": { - "resourceType": "Encounter", - "id": "72b477bd-f07b-4813-a790-779d1d0c10ce", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:eb4af27b-9ad1-4d3e-8392-6bdb9004cd10", - "resource": { - "resourceType": "Observation", - "id": "eb4af27b-9ad1-4d3e-8392-6bdb9004cd10", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:03ddccd2-7898-4cc8-8c2c-8ad4c2e8b23a", - "resource": { - "resourceType": "Observation", - "id": "03ddccd2-7898-4cc8-8c2c-8ad4c2e8b23a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 4, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:032a20e6-dd9e-4859-91ef-c9e07bb82d98", - "resource": { - "resourceType": "Observation", - "id": "032a20e6-dd9e-4859-91ef-c9e07bb82d98", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 49.5, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:84a95b87-d3b2-4e38-b3a8-773f301b322a", - "resource": { - "resourceType": "Observation", - "id": "84a95b87-d3b2-4e38-b3a8-773f301b322a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 19.230, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:496cc321-bdd0-4008-98df-12aa7607321c", - "resource": { - "resourceType": "Observation", - "id": "496cc321-bdd0-4008-98df-12aa7607321c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 80, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 121, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:431d7e02-6096-4c38-9bf4-435fe81560aa", - "resource": { - "resourceType": "Observation", - "id": "431d7e02-6096-4c38-9bf4-435fe81560aa", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 62, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ecd44b84-100a-4386-96d6-41d72d57c79a", - "resource": { - "resourceType": "Observation", - "id": "ecd44b84-100a-4386-96d6-41d72d57c79a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d8b1d755-363c-4f26-a223-debe4c5b636f", - "resource": { - "resourceType": "Observation", - "id": "d8b1d755-363c-4f26-a223-debe4c5b636f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "effectiveDateTime": "2014-11-11T15:24:16-08:00", - "issued": "2014-11-11T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc", - "resource": { - "resourceType": "Procedure", - "id": "c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "performedPeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:42:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:95a13d49-ad21-492d-92c4-07a2bf99f7c1", - "resource": { - "resourceType": "Procedure", - "id": "95a13d49-ad21-492d-92c4-07a2bf99f7c1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "performedPeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9fb1c25d-f3ad-4faf-9773-abb7d0dd575f", - "resource": { - "resourceType": "MedicationRequest", - "id": "9fb1c25d-f3ad-4faf-9773-abb7d0dd575f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - }, - "authoredOn": "2014-11-11T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:ada238df-b638-4073-a820-d854ae53aea0", - "resource": { - "resourceType": "Claim", - "id": "ada238df-b638-4073-a820-d854ae53aea0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:9fb1c25d-f3ad-4faf-9773-abb7d0dd575f" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b74ce645-0edd-4777-b74d-8ea43a007a7f", - "resource": { - "resourceType": "Claim", - "id": "b74ce645-0edd-4777-b74d-8ea43a007a7f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:c6ebd69f-947a-4dc3-9e91-14fcb93d4ecc" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:95a13d49-ad21-492d-92c4-07a2bf99f7c1" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2006.87, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 2 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 679.38, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:37fa0a40-be1e-4fbb-aaaa-dbf56f81d422", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "37fa0a40-be1e-4fbb-aaaa-dbf56f81d422", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b74ce645-0edd-4777-b74d-8ea43a007a7f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2014-11-11T15:57:16-08:00", - "end": "2015-11-11T15:57:16-08:00" - }, - "created": "2014-11-11T15:57:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:b74ce645-0edd-4777-b74d-8ea43a007a7f" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:72b477bd-f07b-4813-a790-779d1d0c10ce" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2006.87, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 401.374, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1605.496, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2006.87, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2006.87, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "servicedPeriod": { - "start": "2014-11-11T15:24:16-08:00", - "end": "2014-11-11T15:57:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 679.38, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 135.876, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 543.504, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 679.38, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 679.38, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2149.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b", - "resource": { - "resourceType": "Encounter", - "id": "0053b566-62ae-4381-8276-dfe8329fa56b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "EMER" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "50849002", - "display": "Emergency room admission (procedure)" - } - ], - "text": "Emergency room admission (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0", - "resource": { - "resourceType": "Condition", - "id": "8943cf36-7c44-4885-8d3f-a42d6cb3d2f0", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "403191005", - "display": "Second degree burn" - } - ], - "text": "Second degree burn" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - }, - "onsetDateTime": "2015-01-10T15:24:16-08:00", - "abatementDateTime": "2015-02-07T15:24:16-08:00", - "assertedDate": "2015-01-10T15:24:16-08:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:19868572-9831-4f32-83a7-a0f4ace4a262", - "resource": { - "resourceType": "MedicationRequest", - "id": "19868572-9831-4f32-83a7-a0f4ace4a262", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "308192", - "display": "Amoxicillin 500 MG Oral Tablet" - } - ], - "text": "Amoxicillin 500 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - }, - "authoredOn": "2015-01-10T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a01b565e-672c-499a-aa3b-bfe1b6d3ad31", - "resource": { - "resourceType": "Claim", - "id": "a01b565e-672c-499a-aa3b-bfe1b6d3ad31", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:19868572-9831-4f32-83a7-a0f4ace4a262" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - } - ] - } - ], - "total": { - "value": 6.39, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:11e723d4-3ad4-45b3-b471-1658db7c1a11", - "resource": { - "resourceType": "MedicationRequest", - "id": "11e723d4-3ad4-45b3-b471-1658db7c1a11", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "861467", - "display": "Meperidine Hydrochloride 50 MG Oral Tablet" - } - ], - "text": "Meperidine Hydrochloride 50 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - }, - "authoredOn": "2015-01-10T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:128b2648-19a7-4f1c-aa7c-bc21c6ba0dec", - "resource": { - "resourceType": "Claim", - "id": "128b2648-19a7-4f1c-aa7c-bc21c6ba0dec", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:11e723d4-3ad4-45b3-b471-1658db7c1a11" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - } - ] - } - ], - "total": { - "value": 63.15, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:9bcbc8c6-32c2-45b6-9771-b0eca73d4eed", - "resource": { - "resourceType": "MedicationRequest", - "id": "9bcbc8c6-32c2-45b6-9771-b0eca73d4eed", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "310965", - "display": "Ibuprofen 200 MG Oral Tablet" - } - ], - "text": "Ibuprofen 200 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - }, - "authoredOn": "2015-01-10T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:57c10579-eadb-475f-a659-bacf204221fc", - "resource": { - "resourceType": "Claim", - "id": "57c10579-eadb-475f-a659-bacf204221fc", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:9bcbc8c6-32c2-45b6-9771-b0eca73d4eed" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - } - ] - } - ], - "total": { - "value": 8.6, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:12bf4c5c-b3d5-43fc-9cf6-40221abe93df", - "resource": { - "resourceType": "CarePlan", - "id": "12bf4c5c-b3d5-43fc-9cf6-40221abe93df", - "status": "completed", - "intent": "order", - "category": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "133901003", - "display": "Burn care" - } - ], - "text": "Burn care" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - }, - "period": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-02-07T15:24:16-08:00" - }, - "addresses": [ - { - "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" - } - ], - "activity": [ - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "385949008", - "display": "Dressing change management" - } - ], - "text": "Dressing change management" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "440381005", - "display": "Behavior to prevent sun exposure" - } - ], - "text": "Behavior to prevent sun exposure" - }, - "status": "completed" - } - }, - { - "detail": { - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "439830001", - "display": "Behavior to prevent infection" - } - ], - "text": "Behavior to prevent infection" - }, - "status": "completed" - } - } - ] - }, - "request": { - "method": "POST", - "url": "CarePlan" - } - }, - { - "fullUrl": "urn:uuid:13aa18f1-796b-4a35-818d-435b8c90eb7e", - "resource": { - "resourceType": "Claim", - "id": "13aa18f1-796b-4a35-818d-435b8c90eb7e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dbe71c77-bdff-4ee7-82fe-fad34fd39e34", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "dbe71c77-bdff-4ee7-82fe-fad34fd39e34", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "13aa18f1-796b-4a35-818d-435b8c90eb7e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-01-10T16:24:16-08:00", - "end": "2016-01-10T16:24:16-08:00" - }, - "created": "2015-01-10T16:24:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:13aa18f1-796b-4a35-818d-435b8c90eb7e" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:8943cf36-7c44-4885-8d3f-a42d6cb3d2f0" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:0053b566-62ae-4381-8276-dfe8329fa56b" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-01-10T15:24:16-08:00", - "end": "2015-01-10T16:24:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "23", - "display": "Emergency Room" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e", - "resource": { - "resourceType": "Encounter", - "id": "5311a30f-25f2-44d6-baed-3db16bea760e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-02-10T15:24:16-08:00", - "end": "2015-02-10T15:49:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b7376f67-61a3-4b5a-9faf-2d33f9f288fd", - "resource": { - "resourceType": "Procedure", - "id": "b7376f67-61a3-4b5a-9faf-2d33f9f288fd", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" - }, - "performedPeriod": { - "start": "2015-02-10T15:24:16-08:00", - "end": "2015-02-10T15:34:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:e0a27f42-4002-4210-bc54-2938a529e4de", - "resource": { - "resourceType": "Claim", - "id": "e0a27f42-4002-4210-bc54-2938a529e4de", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-02-10T15:24:16-08:00", - "end": "2015-02-10T15:49:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b7376f67-61a3-4b5a-9faf-2d33f9f288fd" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 1595.30, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:dc8dbd3c-95d8-4fdd-8f52-e80c05d02005", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "dc8dbd3c-95d8-4fdd-8f52-e80c05d02005", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "e0a27f42-4002-4210-bc54-2938a529e4de" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-02-10T15:49:16-08:00", - "end": "2016-02-10T15:49:16-08:00" - }, - "created": "2015-02-10T15:49:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:e0a27f42-4002-4210-bc54-2938a529e4de" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-02-10T15:24:16-08:00", - "end": "2015-02-10T15:49:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:5311a30f-25f2-44d6-baed-3db16bea760e" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2015-02-10T15:24:16-08:00", - "end": "2015-02-10T15:49:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 1595.30, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 319.06, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1276.24, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 1595.30, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 1595.30, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1276.24, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c99a397c-4270-4682-b9c6-9d41fb8d103e", - "resource": { - "resourceType": "Encounter", - "id": "c99a397c-4270-4682-b9c6-9d41fb8d103e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for 'check-up'" - } - ], - "text": "Encounter for 'check-up'" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-02-07T15:24:16-08:00", - "end": "2015-02-07T15:39:16-08:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "403191005", - "display": "Second degree burn" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:01d74f6b-c8d5-47e8-a3d9-4cde87513cc0", - "resource": { - "resourceType": "Claim", - "id": "01d74f6b-c8d5-47e8-a3d9-4cde87513cc0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-02-07T15:24:16-08:00", - "end": "2015-02-07T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c99a397c-4270-4682-b9c6-9d41fb8d103e" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:6bb085f1-4b6d-45bb-aef9-bb91dfc02c33", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "6bb085f1-4b6d-45bb-aef9-bb91dfc02c33", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "01d74f6b-c8d5-47e8-a3d9-4cde87513cc0" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-02-07T15:39:16-08:00", - "end": "2016-02-07T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99", - "resource": { - "resourceType": "Encounter", - "id": "836c52b1-575c-4bae-8128-68161b5edd99", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-05-12T16:24:16-07:00", - "end": "2015-05-12T16:57:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9", - "resource": { - "resourceType": "Procedure", - "id": "3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" - }, - "performedPeriod": { - "start": "2015-05-12T16:24:16-07:00", - "end": "2015-05-12T16:42:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:c5ee4911-c7e1-4706-b691-861b38c7b7aa", - "resource": { - "resourceType": "Claim", - "id": "c5ee4911-c7e1-4706-b691-861b38c7b7aa", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-05-12T16:24:16-07:00", - "end": "2015-05-12T16:57:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:3cf2b2d7-9692-4ac2-9a69-f7478c5b45f9" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2063.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b458ca66-54b6-498b-ba2e-bb2e4b9a85b8", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b458ca66-54b6-498b-ba2e-bb2e4b9a85b8", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "c5ee4911-c7e1-4706-b691-861b38c7b7aa" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-05-12T16:57:16-07:00", - "end": "2016-05-12T16:57:16-07:00" - }, - "created": "2015-05-12T16:57:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:c5ee4911-c7e1-4706-b691-861b38c7b7aa" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-05-12T16:24:16-07:00", - "end": "2015-05-12T16:57:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:836c52b1-575c-4bae-8128-68161b5edd99" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2015-05-12T16:24:16-07:00", - "end": "2015-05-12T16:57:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2063.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 412.71400000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1650.8560000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2063.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2063.57, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1650.8560000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a", - "resource": { - "resourceType": "Encounter", - "id": "b4176cb3-6141-47ac-b482-802189b2436a", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-08-11T16:24:16-07:00", - "end": "2015-08-11T16:49:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a6316a8f-a8cd-4b87-a952-d1652bad4bb3", - "resource": { - "resourceType": "Procedure", - "id": "a6316a8f-a8cd-4b87-a952-d1652bad4bb3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" - }, - "performedPeriod": { - "start": "2015-08-11T16:24:16-07:00", - "end": "2015-08-11T16:34:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4d86806d-84f3-4543-815a-13164812a981", - "resource": { - "resourceType": "Claim", - "id": "4d86806d-84f3-4543-815a-13164812a981", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-08-11T16:24:16-07:00", - "end": "2015-08-11T16:49:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a6316a8f-a8cd-4b87-a952-d1652bad4bb3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2465.00, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:8abe3532-c650-407d-9fe7-4c0ba073557f", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "8abe3532-c650-407d-9fe7-4c0ba073557f", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4d86806d-84f3-4543-815a-13164812a981" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-08-11T16:49:16-07:00", - "end": "2016-08-11T16:49:16-07:00" - }, - "created": "2015-08-11T16:49:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:4d86806d-84f3-4543-815a-13164812a981" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-08-11T16:24:16-07:00", - "end": "2015-08-11T16:49:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b4176cb3-6141-47ac-b482-802189b2436a" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2015-08-11T16:24:16-07:00", - "end": "2015-08-11T16:49:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2465.00, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 493.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1972.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2465.00, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2465.00, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1972.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6", - "resource": { - "resourceType": "Encounter", - "id": "f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2015-11-10T15:24:16-08:00", - "end": "2015-11-10T15:52:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:7377f015-4487-4c20-b347-1b5a0cd61530", - "resource": { - "resourceType": "Procedure", - "id": "7377f015-4487-4c20-b347-1b5a0cd61530", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" - }, - "performedPeriod": { - "start": "2015-11-10T15:24:16-08:00", - "end": "2015-11-10T15:37:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f76aa0d7-6f99-488d-bde3-bcec1bb5911d", - "resource": { - "resourceType": "Claim", - "id": "f76aa0d7-6f99-488d-bde3-bcec1bb5911d", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-11-10T15:24:16-08:00", - "end": "2015-11-10T15:52:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7377f015-4487-4c20-b347-1b5a0cd61530" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2859.59, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:95c1e424-54a2-4b85-9611-870f06608dcc", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "95c1e424-54a2-4b85-9611-870f06608dcc", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f76aa0d7-6f99-488d-bde3-bcec1bb5911d" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-11-10T15:52:16-08:00", - "end": "2016-11-10T15:52:16-08:00" - }, - "created": "2015-11-10T15:52:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:f76aa0d7-6f99-488d-bde3-bcec1bb5911d" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2015-11-10T15:24:16-08:00", - "end": "2015-11-10T15:52:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:f9e2f3b0-7be6-484d-b966-5ac0a6e4b6c6" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2015-11-10T15:24:16-08:00", - "end": "2015-11-10T15:52:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2859.59, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 571.918, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2287.672, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2859.59, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2859.59, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2287.672, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256", - "resource": { - "resourceType": "Encounter", - "id": "27b6fd4a-2a0e-4d82-a548-65d42540a256", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2015-11-17T15:24:16-08:00", - "end": "2015-11-17T15:54:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:56b7aea8-24fe-4fa5-aa4f-6a341129382a", - "resource": { - "resourceType": "Observation", - "id": "56b7aea8-24fe-4fa5-aa4f-6a341129382a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:921032c0-0114-4a5e-bbcd-0bf1c9bff709", - "resource": { - "resourceType": "Observation", - "id": "921032c0-0114-4a5e-bbcd-0bf1c9bff709", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:a5ef5443-2e42-4e09-ab85-df0c8135a613", - "resource": { - "resourceType": "Observation", - "id": "a5ef5443-2e42-4e09-ab85-df0c8135a613", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 51.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f38dcd03-99ee-4911-9507-722f574c5c15", - "resource": { - "resourceType": "Observation", - "id": "f38dcd03-99ee-4911-9507-722f574c5c15", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 19.940, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ed0e579a-37f5-40d6-a0b2-2e4e79d2980a", - "resource": { - "resourceType": "Observation", - "id": "ed0e579a-37f5-40d6-a0b2-2e4e79d2980a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 81, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 114, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e3246862-7a48-46e9-b4d7-884269f385c8", - "resource": { - "resourceType": "Observation", - "id": "e3246862-7a48-46e9-b4d7-884269f385c8", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 86, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0f3887f9-25d9-4779-bd9c-227a020e2156", - "resource": { - "resourceType": "Observation", - "id": "0f3887f9-25d9-4779-bd9c-227a020e2156", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 16, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", - "resource": { - "resourceType": "Observation", - "id": "76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "6690-2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Leukocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 6.2154, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:58080587-be34-4087-8da5-a0b5613a3e89", - "resource": { - "resourceType": "Observation", - "id": "58080587-be34-4087-8da5-a0b5613a3e89", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "789-8", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - } - ], - "text": "Erythrocytes [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 5.1053, - "unit": "10*6/uL", - "system": "http://unitsofmeasure.org", - "code": "10*6/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6510c26-22f1-4f8a-9771-61136a99039e", - "resource": { - "resourceType": "Observation", - "id": "c6510c26-22f1-4f8a-9771-61136a99039e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "718-7", - "display": "Hemoglobin [Mass/volume] in Blood" - } - ], - "text": "Hemoglobin [Mass/volume] in Blood" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 14.489, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c2ede517-f03d-4666-af16-bc6348570a06", - "resource": { - "resourceType": "Observation", - "id": "c2ede517-f03d-4666-af16-bc6348570a06", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "4544-3", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - } - ], - "text": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 45.437, - "unit": "%", - "system": "http://unitsofmeasure.org", - "code": "%" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:ee94ae44-9dde-459d-835e-568706fb71f5", - "resource": { - "resourceType": "Observation", - "id": "ee94ae44-9dde-459d-835e-568706fb71f5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "787-2", - "display": "MCV [Entitic volume] by Automated count" - } - ], - "text": "MCV [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 87.474, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:54220c6f-3fc7-49f3-a821-aff81184ed40", - "resource": { - "resourceType": "Observation", - "id": "54220c6f-3fc7-49f3-a821-aff81184ed40", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "785-6", - "display": "MCH [Entitic mass] by Automated count" - } - ], - "text": "MCH [Entitic mass] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 31.763, - "unit": "pg", - "system": "http://unitsofmeasure.org", - "code": "pg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:cd828386-be7e-4a31-9e67-06288795eade", - "resource": { - "resourceType": "Observation", - "id": "cd828386-be7e-4a31-9e67-06288795eade", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "786-4", - "display": "MCHC [Mass/volume] by Automated count" - } - ], - "text": "MCHC [Mass/volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 34.857, - "unit": "g/dL", - "system": "http://unitsofmeasure.org", - "code": "g/dL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:accf7740-d4c2-4546-81a6-eed772a2b91b", - "resource": { - "resourceType": "Observation", - "id": "accf7740-d4c2-4546-81a6-eed772a2b91b", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "21000-5", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - } - ], - "text": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 42.645, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7cd987ea-01e1-45bb-a84a-2bf12caa7595", - "resource": { - "resourceType": "Observation", - "id": "7cd987ea-01e1-45bb-a84a-2bf12caa7595", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "777-3", - "display": "Platelets [#/volume] in Blood by Automated count" - } - ], - "text": "Platelets [#/volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 280.68, - "unit": "10*3/uL", - "system": "http://unitsofmeasure.org", - "code": "10*3/uL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:548fcef3-5aeb-4aa9-9b6b-f30aee35c380", - "resource": { - "resourceType": "Observation", - "id": "548fcef3-5aeb-4aa9-9b6b-f30aee35c380", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32207-3", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 340.44, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:61153ae3-8d67-4ee3-ab0a-9173084a8bd3", - "resource": { - "resourceType": "Observation", - "id": "61153ae3-8d67-4ee3-ab0a-9173084a8bd3", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "laboratory", - "display": "laboratory" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "32623-1", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ], - "text": "Platelet mean volume [Entitic volume] in Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueQuantity": { - "value": 9.5791, - "unit": "fL", - "system": "http://unitsofmeasure.org", - "code": "fL" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d1cfb57a-13d9-4773-b78c-4756dbc75cd5", - "resource": { - "resourceType": "Observation", - "id": "d1cfb57a-13d9-4773-b78c-4756dbc75cd5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7c54e167-8da3-427f-a62c-14691981f880", - "resource": { - "resourceType": "Procedure", - "id": "7c54e167-8da3-427f-a62c-14691981f880", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "performedPeriod": { - "start": "2015-11-17T15:24:16-08:00", - "end": "2015-11-17T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f6787a2a-9b22-4cb7-8309-de3dd002a848", - "resource": { - "resourceType": "MedicationRequest", - "id": "f6787a2a-9b22-4cb7-8309-de3dd002a848", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "authoredOn": "2015-11-17T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:7045b927-3b8e-4db5-9f50-981466a611e1", - "resource": { - "resourceType": "Claim", - "id": "7045b927-3b8e-4db5-9f50-981466a611e1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-11-17T15:24:16-08:00", - "end": "2015-11-17T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:f6787a2a-9b22-4cb7-8309-de3dd002a848" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:450ff0e1-86e6-41ad-8abf-eea174649235", - "resource": { - "resourceType": "Immunization", - "id": "450ff0e1-86e6-41ad-8abf-eea174649235", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "date": "2015-11-17T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:52c017da-3670-4ee4-8d3b-8cd949c53b99", - "resource": { - "resourceType": "Immunization", - "id": "52c017da-3670-4ee4-8d3b-8cd949c53b99", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "113", - "display": "Td (adult) preservative free" - } - ], - "text": "Td (adult) preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "date": "2015-11-17T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:20ecd3fa-f7b2-4eb7-b40a-0f344f060d04", - "resource": { - "resourceType": "Immunization", - "id": "20ecd3fa-f7b2-4eb7-b40a-0f344f060d04", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "43", - "display": "Hep B, adult" - } - ], - "text": "Hep B, adult" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "date": "2015-11-17T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:e226d9f5-cfe7-4ad6-a96b-efd5deb61440", - "resource": { - "resourceType": "Immunization", - "id": "e226d9f5-cfe7-4ad6-a96b-efd5deb61440", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "114", - "display": "meningococcal MCV4P" - } - ], - "text": "meningococcal MCV4P" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "date": "2015-11-17T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:72ddb507-ac2f-4a74-b1b0-0ef8eb4690be", - "resource": { - "resourceType": "DiagnosticReport", - "id": "72ddb507-ac2f-4a74-b1b0-0ef8eb4690be", - "status": "final", - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "58410-2", - "display": "Complete blood count (hemogram) panel - Blood by Automated count" - } - ], - "text": "Complete blood count (hemogram) panel - Blood by Automated count" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - }, - "effectiveDateTime": "2015-11-17T15:24:16-08:00", - "issued": "2015-11-17T15:24:16.644-08:00", - "result": [ - { - "reference": "urn:uuid:76c54339-d8ab-4f2d-b2cc-e994ecb6caf2", - "display": "Leukocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:58080587-be34-4087-8da5-a0b5613a3e89", - "display": "Erythrocytes [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:c6510c26-22f1-4f8a-9771-61136a99039e", - "display": "Hemoglobin [Mass/volume] in Blood" - }, - { - "reference": "urn:uuid:c2ede517-f03d-4666-af16-bc6348570a06", - "display": "Hematocrit [Volume Fraction] of Blood by Automated count" - }, - { - "reference": "urn:uuid:ee94ae44-9dde-459d-835e-568706fb71f5", - "display": "MCV [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:54220c6f-3fc7-49f3-a821-aff81184ed40", - "display": "MCH [Entitic mass] by Automated count" - }, - { - "reference": "urn:uuid:cd828386-be7e-4a31-9e67-06288795eade", - "display": "MCHC [Mass/volume] by Automated count" - }, - { - "reference": "urn:uuid:accf7740-d4c2-4546-81a6-eed772a2b91b", - "display": "Erythrocyte distribution width [Entitic volume] by Automated count" - }, - { - "reference": "urn:uuid:7cd987ea-01e1-45bb-a84a-2bf12caa7595", - "display": "Platelets [#/volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:548fcef3-5aeb-4aa9-9b6b-f30aee35c380", - "display": "Platelet distribution width [Entitic volume] in Blood by Automated count" - }, - { - "reference": "urn:uuid:61153ae3-8d67-4ee3-ab0a-9173084a8bd3", - "display": "Platelet mean volume [Entitic volume] in Blood by Automated count" - } - ] - }, - "request": { - "method": "POST", - "url": "DiagnosticReport" - } - }, - { - "fullUrl": "urn:uuid:a1c6c4b5-0929-4205-ae1d-d1a13354df0e", - "resource": { - "resourceType": "Claim", - "id": "a1c6c4b5-0929-4205-ae1d-d1a13354df0e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2015-11-17T15:24:16-08:00", - "end": "2015-11-17T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:450ff0e1-86e6-41ad-8abf-eea174649235" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:52c017da-3670-4ee4-8d3b-8cd949c53b99" - } - }, - { - "sequence": 3, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:20ecd3fa-f7b2-4eb7-b40a-0f344f060d04" - } - }, - { - "sequence": 4, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e226d9f5-cfe7-4ad6-a96b-efd5deb61440" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:7c54e167-8da3-427f-a62c-14691981f880" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:27b6fd4a-2a0e-4d82-a548-65d42540a256" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "informationLinkId": [ - 3 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 5, - "informationLinkId": [ - 4 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 6, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 569.72, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:aa3bad79-1435-488d-bc39-c64aee985a93", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "aa3bad79-1435-488d-bc39-c64aee985a93", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "a1c6c4b5-0929-4205-ae1d-d1a13354df0e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2015-11-17T15:54:16-08:00", - "end": "2016-11-17T15:54:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 5, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 6, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 113.94400000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 455.77600000000007, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 569.72, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 569.72, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 905.44, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e", - "resource": { - "resourceType": "Encounter", - "id": "7fa3c314-cfd9-4828-b940-09561aab021e", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2016-02-09T15:24:16-08:00", - "end": "2016-02-09T15:57:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3", - "resource": { - "resourceType": "Procedure", - "id": "4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" - }, - "performedPeriod": { - "start": "2016-02-09T15:24:16-08:00", - "end": "2016-02-09T15:42:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:bc44af79-8721-43c8-b2cd-47f08b4ee564", - "resource": { - "resourceType": "Claim", - "id": "bc44af79-8721-43c8-b2cd-47f08b4ee564", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-02-09T15:24:16-08:00", - "end": "2016-02-09T15:57:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4d30d2fd-f09d-4bae-80fe-2d4d4a8f52d3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2805.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:93b295ea-0e5e-4736-a82b-09e304a69900", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "93b295ea-0e5e-4736-a82b-09e304a69900", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bc44af79-8721-43c8-b2cd-47f08b4ee564" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-02-09T15:57:16-08:00", - "end": "2017-02-09T15:57:16-08:00" - }, - "created": "2016-02-09T15:57:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:bc44af79-8721-43c8-b2cd-47f08b4ee564" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-02-09T15:24:16-08:00", - "end": "2016-02-09T15:57:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:7fa3c314-cfd9-4828-b940-09561aab021e" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2016-02-09T15:24:16-08:00", - "end": "2016-02-09T15:57:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2805.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 561.158, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2244.632, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2805.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2805.79, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2244.632, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733", - "resource": { - "resourceType": "Encounter", - "id": "5eefddac-8c8c-4e83-914a-c463a6bf3733", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2016-05-10T16:24:16-07:00", - "end": "2016-05-10T16:56:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:144c4c1a-24b0-456a-9604-812b5980254a", - "resource": { - "resourceType": "Procedure", - "id": "144c4c1a-24b0-456a-9604-812b5980254a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" - }, - "performedPeriod": { - "start": "2016-05-10T16:24:16-07:00", - "end": "2016-05-10T16:41:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:3d53c6c9-f912-4250-b82a-1b65fe7cafbc", - "resource": { - "resourceType": "Claim", - "id": "3d53c6c9-f912-4250-b82a-1b65fe7cafbc", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-05-10T16:24:16-07:00", - "end": "2016-05-10T16:56:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:144c4c1a-24b0-456a-9604-812b5980254a" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 1575.77, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:fa2f6cb4-4207-4fb9-a1ee-265e95e8dd6c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "fa2f6cb4-4207-4fb9-a1ee-265e95e8dd6c", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "3d53c6c9-f912-4250-b82a-1b65fe7cafbc" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-05-10T16:56:16-07:00", - "end": "2017-05-10T16:56:16-07:00" - }, - "created": "2016-05-10T16:56:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:3d53c6c9-f912-4250-b82a-1b65fe7cafbc" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-05-10T16:24:16-07:00", - "end": "2016-05-10T16:56:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:5eefddac-8c8c-4e83-914a-c463a6bf3733" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2016-05-10T16:24:16-07:00", - "end": "2016-05-10T16:56:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 1575.77, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 315.154, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1260.616, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 1575.77, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 1575.77, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1260.616, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450", - "resource": { - "resourceType": "Encounter", - "id": "9813d114-5d0d-4ce2-859e-60ba0ee58450", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2016-08-09T16:24:16-07:00", - "end": "2016-08-09T16:57:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a3335838-52dc-499f-b85c-0c45567f4bf2", - "resource": { - "resourceType": "Procedure", - "id": "a3335838-52dc-499f-b85c-0c45567f4bf2", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" - }, - "performedPeriod": { - "start": "2016-08-09T16:24:16-07:00", - "end": "2016-08-09T16:42:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e", - "resource": { - "resourceType": "Claim", - "id": "bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-08-09T16:24:16-07:00", - "end": "2016-08-09T16:57:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a3335838-52dc-499f-b85c-0c45567f4bf2" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3977.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1fdca28d-41f6-4f6f-b1e7-ad5d55d314df", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "1fdca28d-41f6-4f6f-b1e7-ad5d55d314df", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-08-09T16:57:16-07:00", - "end": "2017-08-09T16:57:16-07:00" - }, - "created": "2016-08-09T16:57:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:bfdf9bf1-d6ea-49d6-a32f-fa6344e7074e" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-08-09T16:24:16-07:00", - "end": "2016-08-09T16:57:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:9813d114-5d0d-4ce2-859e-60ba0ee58450" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2016-08-09T16:24:16-07:00", - "end": "2016-08-09T16:57:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3977.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 795.578, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3182.312, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3977.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3977.89, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 3182.312, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8", - "resource": { - "resourceType": "Encounter", - "id": "84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2016-11-08T15:24:16-08:00", - "end": "2016-11-08T15:50:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bd84294e-72b9-43d4-9d1c-8d522ea2dbe4", - "resource": { - "resourceType": "Procedure", - "id": "bd84294e-72b9-43d4-9d1c-8d522ea2dbe4", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" - }, - "performedPeriod": { - "start": "2016-11-08T15:24:16-08:00", - "end": "2016-11-08T15:35:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:961295fd-ac28-4058-b1f3-23186fcc2b9b", - "resource": { - "resourceType": "Claim", - "id": "961295fd-ac28-4058-b1f3-23186fcc2b9b", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-11-08T15:24:16-08:00", - "end": "2016-11-08T15:50:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:bd84294e-72b9-43d4-9d1c-8d522ea2dbe4" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 4243.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a591055a-5f87-4a9f-b016-e1e65bffa0b4", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a591055a-5f87-4a9f-b016-e1e65bffa0b4", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "961295fd-ac28-4058-b1f3-23186fcc2b9b" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-11-08T15:50:16-08:00", - "end": "2017-11-08T15:50:16-08:00" - }, - "created": "2016-11-08T15:50:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:961295fd-ac28-4058-b1f3-23186fcc2b9b" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2016-11-08T15:24:16-08:00", - "end": "2016-11-08T15:50:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:84d07d8f-a6ee-4c63-8a86-26fd6aae9bf8" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2016-11-08T15:24:16-08:00", - "end": "2016-11-08T15:50:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 4243.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 848.7, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3394.8, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 4243.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 4243.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 3394.8, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3", - "resource": { - "resourceType": "Encounter", - "id": "e5eb6cb2-888a-4514-850f-e46a29c504d3", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2016-11-22T15:24:16-08:00", - "end": "2016-11-22T15:54:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:cb28939a-d6a3-4a94-b106-13e799e173a5", - "resource": { - "resourceType": "Observation", - "id": "cb28939a-d6a3-4a94-b106-13e799e173a5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:3d65d4b1-2f4a-4df8-8355-1ead6e29c9b1", - "resource": { - "resourceType": "Observation", - "id": "3d65d4b1-2f4a-4df8-8355-1ead6e29c9b1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 1, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bfbbf58b-633a-4dc0-a7ec-460b2b428855", - "resource": { - "resourceType": "Observation", - "id": "bfbbf58b-633a-4dc0-a7ec-460b2b428855", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 53.400, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:fc51eaea-94b5-442a-b87d-f36f7f74b1f0", - "resource": { - "resourceType": "Observation", - "id": "fc51eaea-94b5-442a-b87d-f36f7f74b1f0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 20.720, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b59620b0-65f9-491d-a580-e8705a9df5b5", - "resource": { - "resourceType": "Observation", - "id": "b59620b0-65f9-491d-a580-e8705a9df5b5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 75, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 111, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:8dd5bb0c-07bb-427e-82a5-563a94d56fe5", - "resource": { - "resourceType": "Observation", - "id": "8dd5bb0c-07bb-427e-82a5-563a94d56fe5", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 94, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:39b3b48a-8113-4fdc-ac46-78240f2bd581", - "resource": { - "resourceType": "Observation", - "id": "39b3b48a-8113-4fdc-ac46-78240f2bd581", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9af92abd-2188-4a61-9139-c9817720dabb", - "resource": { - "resourceType": "Observation", - "id": "9af92abd-2188-4a61-9139-c9817720dabb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "effectiveDateTime": "2016-11-22T15:24:16-08:00", - "issued": "2016-11-22T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:f61c567d-61f5-4861-b182-b7572c2afcc5", - "resource": { - "resourceType": "Procedure", - "id": "f61c567d-61f5-4861-b182-b7572c2afcc5", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "performedPeriod": { - "start": "2016-11-22T15:24:16-08:00", - "end": "2016-11-22T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:7689a885-feeb-4a35-8dc8-f1f33e505142", - "resource": { - "resourceType": "MedicationRequest", - "id": "7689a885-feeb-4a35-8dc8-f1f33e505142", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "authoredOn": "2016-11-22T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:0f583b17-b21f-4c3c-ae68-87a36d2030b9", - "resource": { - "resourceType": "Claim", - "id": "0f583b17-b21f-4c3c-ae68-87a36d2030b9", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-11-22T15:24:16-08:00", - "end": "2016-11-22T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:7689a885-feeb-4a35-8dc8-f1f33e505142" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e632095a-f912-4a53-b609-67cded28f427", - "resource": { - "resourceType": "Immunization", - "id": "e632095a-f912-4a53-b609-67cded28f427", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "date": "2016-11-22T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:cbab968d-6970-468e-be4a-85ce1d351666", - "resource": { - "resourceType": "Immunization", - "id": "cbab968d-6970-468e-be4a-85ce1d351666", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "43", - "display": "Hep B, adult" - } - ], - "text": "Hep B, adult" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - }, - "date": "2016-11-22T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:ab8ebec2-c070-4d1d-8b08-0ce46cb970a4", - "resource": { - "resourceType": "Claim", - "id": "ab8ebec2-c070-4d1d-8b08-0ce46cb970a4", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2016-11-22T15:24:16-08:00", - "end": "2016-11-22T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:e632095a-f912-4a53-b609-67cded28f427" - } - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:cbab968d-6970-468e-be4a-85ce1d351666" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:f61c567d-61f5-4861-b182-b7572c2afcc5" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e5eb6cb2-888a-4514-850f-e46a29c504d3" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "informationLinkId": [ - 2 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 594.99, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a7864301-dcde-469a-acde-5e4a7d4e6a9e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a7864301-dcde-469a-acde-5e4a7d4e6a9e", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "ab8ebec2-c070-4d1d-8b08-0ce46cb970a4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2016-11-22T15:54:16-08:00", - "end": "2017-11-22T15:54:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 118.998, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 475.992, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 594.99, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 594.99, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 700.8240000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67", - "resource": { - "resourceType": "Encounter", - "id": "b80442d6-40f5-4adf-beb1-fc3d30d1fe67", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2017-02-07T15:24:16-08:00", - "end": "2017-02-07T15:58:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d29492cf-53cf-42f4-b9f3-57b0c3c2015c", - "resource": { - "resourceType": "Procedure", - "id": "d29492cf-53cf-42f4-b9f3-57b0c3c2015c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" - }, - "performedPeriod": { - "start": "2017-02-07T15:24:16-08:00", - "end": "2017-02-07T15:43:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9c4d60e1-87e9-49dc-8f48-9362358300ac", - "resource": { - "resourceType": "Claim", - "id": "9c4d60e1-87e9-49dc-8f48-9362358300ac", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-02-07T15:24:16-08:00", - "end": "2017-02-07T15:58:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:d29492cf-53cf-42f4-b9f3-57b0c3c2015c" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3400.19, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:79ebda33-0670-40cf-90e8-10c0f1f26022", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "79ebda33-0670-40cf-90e8-10c0f1f26022", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9c4d60e1-87e9-49dc-8f48-9362358300ac" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-02-07T15:58:16-08:00", - "end": "2018-02-07T15:58:16-08:00" - }, - "created": "2017-02-07T15:58:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:9c4d60e1-87e9-49dc-8f48-9362358300ac" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2017-02-07T15:24:16-08:00", - "end": "2017-02-07T15:58:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b80442d6-40f5-4adf-beb1-fc3d30d1fe67" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2017-02-07T15:24:16-08:00", - "end": "2017-02-07T15:58:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3400.19, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 680.038, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2720.152, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3400.19, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3400.19, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2720.152, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393", - "resource": { - "resourceType": "Encounter", - "id": "615d0f10-3f62-42c8-9fa3-400bc8adf393", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2017-05-09T16:24:16-07:00", - "end": "2017-05-09T16:50:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:529092e3-aef8-4964-bb61-a90be792e125", - "resource": { - "resourceType": "Procedure", - "id": "529092e3-aef8-4964-bb61-a90be792e125", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" - }, - "performedPeriod": { - "start": "2017-05-09T16:24:16-07:00", - "end": "2017-05-09T16:35:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:aa3f48d5-4407-40af-b52e-dd2e43ade642", - "resource": { - "resourceType": "Claim", - "id": "aa3f48d5-4407-40af-b52e-dd2e43ade642", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-05-09T16:24:16-07:00", - "end": "2017-05-09T16:50:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:529092e3-aef8-4964-bb61-a90be792e125" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3045.69, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d84ead3f-a1fb-4ce6-9fe9-0a170dc5dd36", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "d84ead3f-a1fb-4ce6-9fe9-0a170dc5dd36", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "aa3f48d5-4407-40af-b52e-dd2e43ade642" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-05-09T16:50:16-07:00", - "end": "2018-05-09T16:50:16-07:00" - }, - "created": "2017-05-09T16:50:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:aa3f48d5-4407-40af-b52e-dd2e43ade642" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2017-05-09T16:24:16-07:00", - "end": "2017-05-09T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:615d0f10-3f62-42c8-9fa3-400bc8adf393" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2017-05-09T16:24:16-07:00", - "end": "2017-05-09T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3045.69, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 609.138, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2436.552, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3045.69, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3045.69, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2436.552, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf", - "resource": { - "resourceType": "Encounter", - "id": "e541275d-1165-4ae1-8cac-0cba7645c1cf", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:54:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5cc75830-38bd-43fd-bf7a-56a655e3f502", - "resource": { - "resourceType": "Procedure", - "id": "5cc75830-38bd-43fd-bf7a-56a655e3f502", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" - }, - "performedPeriod": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:39:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f59c2d32-a435-4454-82c6-71f297a11c3b", - "resource": { - "resourceType": "MedicationRequest", - "id": "f59c2d32-a435-4454-82c6-71f297a11c3b", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "1000126", - "display": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" - } - ], - "text": "1 ML medroxyPROGESTERone acetate 150 MG/ML Injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" - }, - "authoredOn": "2017-07-17T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a559500c-b320-4ade-baca-39004ef8cc5f", - "resource": { - "resourceType": "Claim", - "id": "a559500c-b320-4ade-baca-39004ef8cc5f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:54:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:f59c2d32-a435-4454-82c6-71f297a11c3b" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" - } - ] - } - ], - "total": { - "value": 142.94, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:d7bf818e-1745-4237-9837-fb165ca6a864", - "resource": { - "resourceType": "Claim", - "id": "d7bf818e-1745-4237-9837-fb165ca6a864", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:54:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:5cc75830-38bd-43fd-bf7a-56a655e3f502" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2692.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:751b75ec-a610-4a43-95fe-2ab988d83746", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "751b75ec-a610-4a43-95fe-2ab988d83746", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "d7bf818e-1745-4237-9837-fb165ca6a864" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-07-17T16:54:16-07:00", - "end": "2018-07-17T16:54:16-07:00" - }, - "created": "2017-07-17T16:54:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:d7bf818e-1745-4237-9837-fb165ca6a864" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:54:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e541275d-1165-4ae1-8cac-0cba7645c1cf" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2017-07-17T16:24:16-07:00", - "end": "2017-07-17T16:54:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2692.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 538.5500000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2154.2000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2692.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2692.75, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2154.2000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111", - "resource": { - "resourceType": "Encounter", - "id": "e20727ff-80bb-4eb7-86c9-72397dcdb111", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2017-08-08T16:24:16-07:00", - "end": "2017-08-08T16:55:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:75e4692b-b185-431f-8925-5f2218b91b46", - "resource": { - "resourceType": "Procedure", - "id": "75e4692b-b185-431f-8925-5f2218b91b46", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" - }, - "performedPeriod": { - "start": "2017-08-08T16:24:16-07:00", - "end": "2017-08-08T16:40:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:99702b89-b1e1-48ed-a6a4-fdcbd52bcfec", - "resource": { - "resourceType": "Claim", - "id": "99702b89-b1e1-48ed-a6a4-fdcbd52bcfec", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-08-08T16:24:16-07:00", - "end": "2017-08-08T16:55:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:75e4692b-b185-431f-8925-5f2218b91b46" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 4099.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:a27f37ad-cb21-4eb1-a60b-4b92a224adeb", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "a27f37ad-cb21-4eb1-a60b-4b92a224adeb", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "99702b89-b1e1-48ed-a6a4-fdcbd52bcfec" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-08-08T16:55:16-07:00", - "end": "2018-08-08T16:55:16-07:00" - }, - "created": "2017-08-08T16:55:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:99702b89-b1e1-48ed-a6a4-fdcbd52bcfec" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2017-08-08T16:24:16-07:00", - "end": "2017-08-08T16:55:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:e20727ff-80bb-4eb7-86c9-72397dcdb111" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2017-08-08T16:24:16-07:00", - "end": "2017-08-08T16:55:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 4099.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 819.884, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3279.536, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 4099.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 4099.42, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 3279.536, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7", - "resource": { - "resourceType": "Encounter", - "id": "8a77baac-ce39-4c76-9ac3-57e239fd23b7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2017-11-07T15:24:16-08:00", - "end": "2017-11-07T15:50:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:96ad8284-b274-40dd-a901-09da986d98c1", - "resource": { - "resourceType": "Procedure", - "id": "96ad8284-b274-40dd-a901-09da986d98c1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" - }, - "performedPeriod": { - "start": "2017-11-07T15:24:16-08:00", - "end": "2017-11-07T15:35:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:742d2a7d-cfa5-461e-9108-9c76a1d02850", - "resource": { - "resourceType": "Claim", - "id": "742d2a7d-cfa5-461e-9108-9c76a1d02850", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-11-07T15:24:16-08:00", - "end": "2017-11-07T15:50:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:96ad8284-b274-40dd-a901-09da986d98c1" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2659.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:0923210a-7eb1-4cc5-874e-4bfdeb6a3e7d", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "0923210a-7eb1-4cc5-874e-4bfdeb6a3e7d", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "742d2a7d-cfa5-461e-9108-9c76a1d02850" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-11-07T15:50:16-08:00", - "end": "2018-11-07T15:50:16-08:00" - }, - "created": "2017-11-07T15:50:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:742d2a7d-cfa5-461e-9108-9c76a1d02850" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2017-11-07T15:24:16-08:00", - "end": "2017-11-07T15:50:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:8a77baac-ce39-4c76-9ac3-57e239fd23b7" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2017-11-07T15:24:16-08:00", - "end": "2017-11-07T15:50:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2659.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 531.9399999999999, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2127.7599999999998, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2659.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2659.70, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2127.7599999999998, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534", - "resource": { - "resourceType": "Encounter", - "id": "c9a1750e-ebd2-47ea-ba82-07d15a39b534", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2017-11-28T15:24:16-08:00", - "end": "2017-11-28T15:54:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:eb5d23e2-00d6-494f-87f5-8b656648f32f", - "resource": { - "resourceType": "Observation", - "id": "eb5d23e2-00d6-494f-87f5-8b656648f32f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:0f750612-e979-4f7d-bf82-787f0c243e7d", - "resource": { - "resourceType": "Observation", - "id": "0f750612-e979-4f7d-bf82-787f0c243e7d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 0, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e015abf0-3f22-4cbe-bcde-8e6212e70955", - "resource": { - "resourceType": "Observation", - "id": "e015abf0-3f22-4cbe-bcde-8e6212e70955", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 54.800, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:32c5313c-c85d-4854-b8a0-9cc0d0c7c806", - "resource": { - "resourceType": "Observation", - "id": "32c5313c-c85d-4854-b8a0-9cc0d0c7c806", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 21.280, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:20bee56c-749a-4186-bc51-baf252fa2269", - "resource": { - "resourceType": "Observation", - "id": "20bee56c-749a-4186-bc51-baf252fa2269", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 88, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7abf6374-0c6d-4ff6-803b-581b89dbb400", - "resource": { - "resourceType": "Observation", - "id": "7abf6374-0c6d-4ff6-803b-581b89dbb400", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 79, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b37857cb-629a-4f48-bd63-45434d614f4f", - "resource": { - "resourceType": "Observation", - "id": "b37857cb-629a-4f48-bd63-45434d614f4f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:d6b6b044-8a6b-40e1-9507-beee3861f286", - "resource": { - "resourceType": "Observation", - "id": "d6b6b044-8a6b-40e1-9507-beee3861f286", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "effectiveDateTime": "2017-11-28T15:24:16-08:00", - "issued": "2017-11-28T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e6cfb446-a7cf-4a8f-b9ff-785266b3723a", - "resource": { - "resourceType": "Procedure", - "id": "e6cfb446-a7cf-4a8f-b9ff-785266b3723a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "performedPeriod": { - "start": "2017-11-28T15:24:16-08:00", - "end": "2017-11-28T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:96c51039-d8ec-401e-b298-c7e2883d1803", - "resource": { - "resourceType": "MedicationRequest", - "id": "96c51039-d8ec-401e-b298-c7e2883d1803", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "authoredOn": "2017-11-28T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:05f25804-496e-4831-9e06-ea2615b220e0", - "resource": { - "resourceType": "Claim", - "id": "05f25804-496e-4831-9e06-ea2615b220e0", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-11-28T15:24:16-08:00", - "end": "2017-11-28T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:96c51039-d8ec-401e-b298-c7e2883d1803" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:52069ac5-bab8-4ae7-849a-89dd783006c9", - "resource": { - "resourceType": "Immunization", - "id": "52069ac5-bab8-4ae7-849a-89dd783006c9", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - }, - "date": "2017-11-28T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:9cac16f9-504c-4e79-a964-9aaf2af3d090", - "resource": { - "resourceType": "Claim", - "id": "9cac16f9-504c-4e79-a964-9aaf2af3d090", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2017-11-28T15:24:16-08:00", - "end": "2017-11-28T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:52069ac5-bab8-4ae7-849a-89dd783006c9" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e6cfb446-a7cf-4a8f-b9ff-785266b3723a" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c9a1750e-ebd2-47ea-ba82-07d15a39b534" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 828.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:e9b01422-3316-40cf-98a2-3f2e3fa294fe", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "e9b01422-3316-40cf-98a2-3f2e3fa294fe", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "9cac16f9-504c-4e79-a964-9aaf2af3d090" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2017-11-28T15:54:16-08:00", - "end": "2018-11-28T15:54:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 165.74800000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 662.9920000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 828.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 828.74, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 775.4080000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086", - "resource": { - "resourceType": "Encounter", - "id": "57d221cf-bfb3-4fa3-9bf9-684607766086", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2018-02-06T15:24:16-08:00", - "end": "2018-02-06T15:54:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:e64c2f4d-e72e-41d9-9b28-5c1519ee1641", - "resource": { - "resourceType": "Procedure", - "id": "e64c2f4d-e72e-41d9-9b28-5c1519ee1641", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" - }, - "performedPeriod": { - "start": "2018-02-06T15:24:16-08:00", - "end": "2018-02-06T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:b82ea338-9d3e-4346-a7e3-6e2d90607458", - "resource": { - "resourceType": "Claim", - "id": "b82ea338-9d3e-4346-a7e3-6e2d90607458", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-02-06T15:24:16-08:00", - "end": "2018-02-06T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:e64c2f4d-e72e-41d9-9b28-5c1519ee1641" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2069.04, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:c16b4bd1-b007-48c1-91b0-169ad84b331c", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "c16b4bd1-b007-48c1-91b0-169ad84b331c", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b82ea338-9d3e-4346-a7e3-6e2d90607458" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-02-06T15:54:16-08:00", - "end": "2019-02-06T15:54:16-08:00" - }, - "created": "2018-02-06T15:54:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:b82ea338-9d3e-4346-a7e3-6e2d90607458" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-02-06T15:24:16-08:00", - "end": "2018-02-06T15:54:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:57d221cf-bfb3-4fa3-9bf9-684607766086" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2018-02-06T15:24:16-08:00", - "end": "2018-02-06T15:54:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2069.04, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 413.808, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1655.232, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2069.04, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2069.04, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1655.232, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b", - "resource": { - "resourceType": "Encounter", - "id": "1f186cb0-f553-455d-bed0-dbe4a4bb627b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2018-05-08T16:24:16-07:00", - "end": "2018-05-08T16:58:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:b1ed392b-a682-4123-8c6f-e5615cf0b359", - "resource": { - "resourceType": "Procedure", - "id": "b1ed392b-a682-4123-8c6f-e5615cf0b359", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" - }, - "performedPeriod": { - "start": "2018-05-08T16:24:16-07:00", - "end": "2018-05-08T16:43:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:db932c55-16f3-4e3a-aa0d-4325537c066e", - "resource": { - "resourceType": "Claim", - "id": "db932c55-16f3-4e3a-aa0d-4325537c066e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-05-08T16:24:16-07:00", - "end": "2018-05-08T16:58:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b1ed392b-a682-4123-8c6f-e5615cf0b359" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3009.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:b6b44ca7-ecba-439a-b910-342cefa121f0", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "b6b44ca7-ecba-439a-b910-342cefa121f0", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "db932c55-16f3-4e3a-aa0d-4325537c066e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-05-08T16:58:16-07:00", - "end": "2019-05-08T16:58:16-07:00" - }, - "created": "2018-05-08T16:58:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:db932c55-16f3-4e3a-aa0d-4325537c066e" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-05-08T16:24:16-07:00", - "end": "2018-05-08T16:58:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1f186cb0-f553-455d-bed0-dbe4a4bb627b" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2018-05-08T16:24:16-07:00", - "end": "2018-05-08T16:58:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3009.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 601.902, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2407.608, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3009.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3009.51, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2407.608, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580", - "resource": { - "resourceType": "Encounter", - "id": "1781a84e-5949-4dfb-9dfd-40a48141f580", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2018-08-07T16:24:16-07:00", - "end": "2018-08-07T16:58:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c3ff2f9d-fca8-4805-92df-6aecfa57de7b", - "resource": { - "resourceType": "Procedure", - "id": "c3ff2f9d-fca8-4805-92df-6aecfa57de7b", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" - }, - "performedPeriod": { - "start": "2018-08-07T16:24:16-07:00", - "end": "2018-08-07T16:43:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:61fde201-fafd-43a5-8b0f-5b224768af57", - "resource": { - "resourceType": "Claim", - "id": "61fde201-fafd-43a5-8b0f-5b224768af57", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-08-07T16:24:16-07:00", - "end": "2018-08-07T16:58:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:c3ff2f9d-fca8-4805-92df-6aecfa57de7b" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2889.55, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:66f77d02-e7f2-431c-8011-b07e16b0f128", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "66f77d02-e7f2-431c-8011-b07e16b0f128", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "61fde201-fafd-43a5-8b0f-5b224768af57" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-08-07T16:58:16-07:00", - "end": "2019-08-07T16:58:16-07:00" - }, - "created": "2018-08-07T16:58:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:61fde201-fafd-43a5-8b0f-5b224768af57" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-08-07T16:24:16-07:00", - "end": "2018-08-07T16:58:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:1781a84e-5949-4dfb-9dfd-40a48141f580" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2018-08-07T16:24:16-07:00", - "end": "2018-08-07T16:58:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2889.55, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 577.9100000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2311.6400000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2889.55, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2889.55, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2311.6400000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7", - "resource": { - "resourceType": "Encounter", - "id": "b0b85503-3074-400f-a1d6-841125065cd7", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2018-11-06T15:24:16-08:00", - "end": "2018-11-06T15:51:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:a7961d03-f6a4-4527-b9f8-82a3ce43a4f1", - "resource": { - "resourceType": "Procedure", - "id": "a7961d03-f6a4-4527-b9f8-82a3ce43a4f1", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" - }, - "performedPeriod": { - "start": "2018-11-06T15:24:16-08:00", - "end": "2018-11-06T15:36:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:4d2a50d9-dcba-43c1-8f79-4536334ea3fd", - "resource": { - "resourceType": "Claim", - "id": "4d2a50d9-dcba-43c1-8f79-4536334ea3fd", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-11-06T15:24:16-08:00", - "end": "2018-11-06T15:51:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:a7961d03-f6a4-4527-b9f8-82a3ce43a4f1" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2557.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:eeb3c89a-e04a-4bb4-9bfd-8ad97767939e", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "eeb3c89a-e04a-4bb4-9bfd-8ad97767939e", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "4d2a50d9-dcba-43c1-8f79-4536334ea3fd" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-11-06T15:51:16-08:00", - "end": "2019-11-06T15:51:16-08:00" - }, - "created": "2018-11-06T15:51:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:4d2a50d9-dcba-43c1-8f79-4536334ea3fd" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2018-11-06T15:24:16-08:00", - "end": "2018-11-06T15:51:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:b0b85503-3074-400f-a1d6-841125065cd7" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2018-11-06T15:24:16-08:00", - "end": "2018-11-06T15:51:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2557.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 511.58400000000006, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2046.3360000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2557.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2557.92, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2046.3360000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78", - "resource": { - "resourceType": "Encounter", - "id": "5ad6b6ab-bcec-49d0-808e-c621cb071b78", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2018-12-04T15:24:16-08:00", - "end": "2018-12-04T15:54:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:74dcf884-17e7-4201-ae3a-50b726d59399", - "resource": { - "resourceType": "Observation", - "id": "74dcf884-17e7-4201-ae3a-50b726d59399", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:82c9c963-030b-4d88-995f-6a1448e1501f", - "resource": { - "resourceType": "Observation", - "id": "82c9c963-030b-4d88-995f-6a1448e1501f", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:1a92fcbe-7551-4b93-9d5b-a4f3aed0585e", - "resource": { - "resourceType": "Observation", - "id": "1a92fcbe-7551-4b93-9d5b-a4f3aed0585e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 56.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4683224a-c9c1-4e1b-adb2-b57e7d80fd73", - "resource": { - "resourceType": "Observation", - "id": "4683224a-c9c1-4e1b-adb2-b57e7d80fd73", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 21.760, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:686678d7-24ec-4d18-a721-5d0e739be299", - "resource": { - "resourceType": "Observation", - "id": "686678d7-24ec-4d18-a721-5d0e739be299", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 79, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 130, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b5b9a122-bc57-4ecc-b76c-e9ba2011a5dd", - "resource": { - "resourceType": "Observation", - "id": "b5b9a122-bc57-4ecc-b76c-e9ba2011a5dd", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 92, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:dc56f53c-ed89-46f9-9a71-ad205769f5bb", - "resource": { - "resourceType": "Observation", - "id": "dc56f53c-ed89-46f9-9a71-ad205769f5bb", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:11716994-7b0a-4acb-9630-9132acb548c4", - "resource": { - "resourceType": "Observation", - "id": "11716994-7b0a-4acb-9630-9132acb548c4", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "effectiveDateTime": "2018-12-04T15:24:16-08:00", - "issued": "2018-12-04T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b9d8995f-9c1d-4b46-9be8-71a5e85370a7", - "resource": { - "resourceType": "Procedure", - "id": "b9d8995f-9c1d-4b46-9be8-71a5e85370a7", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "430193006", - "display": "Medication Reconciliation (procedure)" - } - ], - "text": "Medication Reconciliation (procedure)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "performedPeriod": { - "start": "2018-12-04T15:24:16-08:00", - "end": "2018-12-04T15:39:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:1ac6052c-2519-4964-b87e-a85881001e5f", - "resource": { - "resourceType": "MedicationRequest", - "id": "1ac6052c-2519-4964-b87e-a85881001e5f", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "authoredOn": "2018-12-04T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:07c2f84b-ec79-4aa0-8ca2-47240774cc8f", - "resource": { - "resourceType": "Claim", - "id": "07c2f84b-ec79-4aa0-8ca2-47240774cc8f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-12-04T15:24:16-08:00", - "end": "2018-12-04T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:1ac6052c-2519-4964-b87e-a85881001e5f" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:1e959667-3a05-4657-8b0b-d0acc8b83a4c", - "resource": { - "resourceType": "Immunization", - "id": "1e959667-3a05-4657-8b0b-d0acc8b83a4c", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - }, - "date": "2018-12-04T15:24:16-08:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9", - "resource": { - "resourceType": "Claim", - "id": "14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2018-12-04T15:24:16-08:00", - "end": "2018-12-04T15:54:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:1e959667-3a05-4657-8b0b-d0acc8b83a4c" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:b9d8995f-9c1d-4b46-9be8-71a5e85370a7" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5ad6b6ab-bcec-49d0-808e-c621cb071b78" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "net": { - "value": 529.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:ad76f367-22da-4413-8d83-fb2ebf1ea597", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "ad76f367-22da-4413-8d83-fb2ebf1ea597", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "14d2c60a-1b4b-495b-88b1-d1d93c1bf7f9" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2018-12-04T15:54:16-08:00", - "end": "2019-12-04T15:54:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "430193006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 105.94600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 423.78400000000005, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 529.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 529.73, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 536.2, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d", - "resource": { - "resourceType": "Encounter", - "id": "ca05aff0-7d51-4589-af05-4e7326ed294d", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-02-05T15:24:16-08:00", - "end": "2019-02-05T15:53:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:32839def-0a11-44ca-bc1f-3f4801db8b06", - "resource": { - "resourceType": "Procedure", - "id": "32839def-0a11-44ca-bc1f-3f4801db8b06", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" - }, - "performedPeriod": { - "start": "2019-02-05T15:24:16-08:00", - "end": "2019-02-05T15:38:16-08:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:f444b593-e10e-4763-acc9-c421fc99cad1", - "resource": { - "resourceType": "Claim", - "id": "f444b593-e10e-4763-acc9-c421fc99cad1", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-02-05T15:24:16-08:00", - "end": "2019-02-05T15:53:16-08:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:32839def-0a11-44ca-bc1f-3f4801db8b06" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 3612.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:359c91c3-b781-415e-ba82-c16c5b35d928", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "359c91c3-b781-415e-ba82-c16c5b35d928", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "f444b593-e10e-4763-acc9-c421fc99cad1" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-02-05T15:53:16-08:00", - "end": "2020-02-05T15:53:16-08:00" - }, - "created": "2019-02-05T15:53:16-08:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:f444b593-e10e-4763-acc9-c421fc99cad1" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-02-05T15:24:16-08:00", - "end": "2019-02-05T15:53:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:ca05aff0-7d51-4589-af05-4e7326ed294d" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2019-02-05T15:24:16-08:00", - "end": "2019-02-05T15:53:16-08:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 3612.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 722.5360000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 2890.1440000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 3612.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 3612.68, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 2890.1440000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478", - "resource": { - "resourceType": "Encounter", - "id": "3a830afb-f2b3-4513-ba44-546507458478", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185349003", - "display": "Encounter for check up (procedure)" - } - ], - "text": "Encounter for check up (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-04-16T16:24:16-07:00", - "end": "2019-04-16T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:d15936cd-4e20-41cd-987a-274987cb0798", - "resource": { - "resourceType": "MedicationRequest", - "id": "d15936cd-4e20-41cd-987a-274987cb0798", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" - }, - "authoredOn": "2019-04-16T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:b4d42175-b63d-4cc9-b1ad-5ff8fb929076", - "resource": { - "resourceType": "Claim", - "id": "b4d42175-b63d-4cc9-b1ad-5ff8fb929076", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-04-16T16:24:16-07:00", - "end": "2019-04-16T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:d15936cd-4e20-41cd-987a-274987cb0798" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:3e1963e2-6a19-45f2-97ed-db1bdc1a4c19", - "resource": { - "resourceType": "Immunization", - "id": "3e1963e2-6a19-45f2-97ed-db1bdc1a4c19", - "status": "completed", - "notGiven": false, - "vaccineCode": { - "coding": [ - { - "system": "http://hl7.org/fhir/sid/cvx", - "code": "140", - "display": "Influenza, seasonal, injectable, preservative free" - } - ], - "text": "Influenza, seasonal, injectable, preservative free" - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "encounter": { - "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" - }, - "date": "2019-04-16T16:24:16-07:00", - "primarySource": true - }, - "request": { - "method": "POST", - "url": "Immunization" - } - }, - { - "fullUrl": "urn:uuid:b0c38318-529d-4f3d-9b78-406a5adcc2e2", - "resource": { - "resourceType": "Claim", - "id": "b0c38318-529d-4f3d-9b78-406a5adcc2e2", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-04-16T16:24:16-07:00", - "end": "2019-04-16T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "information": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "http://hl7.org/fhir/claiminformationcategory", - "code": "info" - } - ] - }, - "valueReference": { - "reference": "urn:uuid:3e1963e2-6a19-45f2-97ed-db1bdc1a4c19" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:424c7c68-99c4-4424-a9f7-181a32af532b", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "424c7c68-99c4-4424-a9f7-181a32af532b", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "b0c38318-529d-4f3d-9b78-406a5adcc2e2" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-04-16T16:39:16-07:00", - "end": "2020-04-16T16:39:16-07:00" - }, - "created": "2019-04-16T16:39:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:b0c38318-529d-4f3d-9b78-406a5adcc2e2" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-04-16T16:24:16-07:00", - "end": "2019-04-16T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:3a830afb-f2b3-4513-ba44-546507458478" - } - ] - }, - { - "sequence": 2, - "informationLinkId": [ - 1 - ], - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-04-16T16:24:16-07:00", - "end": "2019-04-16T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 28.104000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 140.52, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 112.41600000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4", - "resource": { - "resourceType": "Encounter", - "id": "42f30f56-7718-40f0-aaea-4af113c3b0d4", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "185345009", - "display": "Encounter for symptom" - } - ], - "text": "Encounter for symptom" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-04-13T16:24:16-07:00", - "end": "2019-04-13T16:39:16-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3", - "resource": { - "resourceType": "Condition", - "id": "9f73c057-cb43-4696-860d-2c5b3b876af3", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "195662009", - "display": "Acute viral pharyngitis (disorder)" - } - ], - "text": "Acute viral pharyngitis (disorder)" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "onsetDateTime": "2019-04-13T16:24:16-07:00", - "abatementDateTime": "2019-04-21T16:24:16-07:00", - "assertedDate": "2019-04-13T16:24:16-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:b1d1927a-d7f3-4113-a2da-ae978c9bf38d", - "resource": { - "resourceType": "Observation", - "id": "b1d1927a-d7f3-4113-a2da-ae978c9bf38d", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8310-5", - "display": "Body temperature" - } - ], - "text": "Body temperature" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-13T16:24:16-07:00", - "issued": "2019-04-13T16:24:16.644-07:00", - "valueQuantity": { - "value": 37.342, - "unit": "Cel", - "system": "http://unitsofmeasure.org", - "code": "Cel" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:01882e32-4618-4fd9-ae85-80148f8deb70", - "resource": { - "resourceType": "Observation", - "id": "01882e32-4618-4fd9-ae85-80148f8deb70", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:e55a0b1c-0639-433b-8ae4-81a769e287c2", - "resource": { - "resourceType": "Observation", - "id": "e55a0b1c-0639-433b-8ae4-81a769e287c2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 3, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:c6c879d6-88c2-4b76-b930-526d105adff0", - "resource": { - "resourceType": "Observation", - "id": "c6c879d6-88c2-4b76-b930-526d105adff0", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 56.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4c8379ca-3c71-47bc-a58f-a39e52dfe557", - "resource": { - "resourceType": "Observation", - "id": "4c8379ca-3c71-47bc-a58f-a39e52dfe557", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 21.760, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bd8773a8-709c-4ba9-b925-979b503aabc7", - "resource": { - "resourceType": "Observation", - "id": "bd8773a8-709c-4ba9-b925-979b503aabc7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 86, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 123, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:4f5cf999-facf-4070-b6cf-0dd62694c75a", - "resource": { - "resourceType": "Observation", - "id": "4f5cf999-facf-4070-b6cf-0dd62694c75a", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 68, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:19a7a129-4e38-4e3b-9c04-dedc560012f1", - "resource": { - "resourceType": "Observation", - "id": "19a7a129-4e38-4e3b-9c04-dedc560012f1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueQuantity": { - "value": 13, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:7e32bdd2-0c04-4b3d-a3db-7f7466f0f318", - "resource": { - "resourceType": "Observation", - "id": "7e32bdd2-0c04-4b3d-a3db-7f7466f0f318", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "effectiveDateTime": "2019-04-16T16:24:16-07:00", - "issued": "2019-04-16T16:24:16.644-07:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:88506957-24fe-4807-8326-7e2c50a75998", - "resource": { - "resourceType": "MedicationRequest", - "id": "88506957-24fe-4807-8326-7e2c50a75998", - "status": "stopped", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - }, - "authoredOn": "2019-04-16T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:a1aa0150-cc31-4919-b243-8177710d1f1c", - "resource": { - "resourceType": "Claim", - "id": "a1aa0150-cc31-4919-b243-8177710d1f1c", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-04-13T16:24:16-07:00", - "end": "2019-04-13T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:88506957-24fe-4807-8326-7e2c50a75998" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7f1957a1-1522-4d2f-858a-ef575ab635e4", - "resource": { - "resourceType": "Claim", - "id": "7f1957a1-1522-4d2f-858a-ef575ab635e4", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-04-13T16:24:16-07:00", - "end": "2019-04-13T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:42f30f56-7718-40f0-aaea-4af113c3b0d4" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:73a962fd-c8b2-40b2-86a2-3d465e653def", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "73a962fd-c8b2-40b2-86a2-3d465e653def", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "7f1957a1-1522-4d2f-858a-ef575ab635e4" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-04-13T16:39:16-07:00", - "end": "2020-04-13T16:39:16-07:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:9f73c057-cb43-4696-860d-2c5b3b876af3" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce", - "resource": { - "resourceType": "Encounter", - "id": "c5353211-363e-4740-8d26-163cd81779ce", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "308335008", - "display": "Patient encounter procedure" - } - ], - "text": "Patient encounter procedure" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-05-07T16:24:16-07:00", - "end": "2019-05-07T16:50:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:bd16401f-27a3-4e93-875d-848fcc226d0d", - "resource": { - "resourceType": "Procedure", - "id": "bd16401f-27a3-4e93-875d-848fcc226d0d", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "76601001", - "display": "Intramuscular injection" - } - ], - "text": "Intramuscular injection" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" - }, - "performedPeriod": { - "start": "2019-05-07T16:24:16-07:00", - "end": "2019-05-07T16:35:16-07:00" - } - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:daef21e7-4157-4cc9-974b-8bcce6742a94", - "resource": { - "resourceType": "Claim", - "id": "daef21e7-4157-4cc9-974b-8bcce6742a94", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-05-07T16:24:16-07:00", - "end": "2019-05-07T16:50:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:bd16401f-27a3-4e93-875d-848fcc226d0d" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "net": { - "value": 2137.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:54b60604-c463-4a6c-86d3-b1c1cc9219b5", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "54b60604-c463-4a6c-86d3-b1c1cc9219b5", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "daef21e7-4157-4cc9-974b-8bcce6742a94" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-05-07T16:50:16-07:00", - "end": "2020-05-07T16:50:16-07:00" - }, - "created": "2019-05-07T16:50:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:daef21e7-4157-4cc9-974b-8bcce6742a94" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-05-07T16:24:16-07:00", - "end": "2019-05-07T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:c5353211-363e-4740-8d26-163cd81779ce" - } - ] - }, - { - "sequence": 2, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "service": { - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "76601001" - } - ] - }, - "servicedPeriod": { - "start": "2019-05-07T16:24:16-07:00", - "end": "2019-05-07T16:50:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "net": { - "value": 2137.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 427.53400000000005, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 1710.1360000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 2137.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 2137.67, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - }, - "reason": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "code": "A", - "display": "Allowed" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1710.1360000000002, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416", - "resource": { - "resourceType": "Encounter", - "id": "ae6e038e-9e72-45a4-ab30-6f0a24fbd416", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424441002", - "display": "Prenatal initial visit" - } - ], - "text": "Prenatal initial visit" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-08-13T16:24:16-07:00", - "end": "2019-08-13T17:09:16-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "resource": { - "resourceType": "Condition", - "id": "c9731186-6731-4a66-82e2-0669427caeb2", - "clinicalStatus": "resolved", - "verificationStatus": "confirmed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ], - "text": "Normal pregnancy" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" - }, - "onsetDateTime": "2019-08-13T16:24:16-07:00", - "abatementDateTime": "2019-08-27T16:24:16-07:00", - "assertedDate": "2019-08-13T16:24:16-07:00" - }, - "request": { - "method": "POST", - "url": "Condition" - } - }, - { - "fullUrl": "urn:uuid:4d516bf3-2917-46e6-87be-51264b421e30", - "resource": { - "resourceType": "Procedure", - "id": "4d516bf3-2917-46e6-87be-51264b421e30", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "252160004", - "display": "Standard pregnancy test" - } - ], - "text": "Standard pregnancy test" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" - }, - "performedPeriod": { - "start": "2019-08-13T16:24:16-07:00", - "end": "2019-08-13T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:9f541dc0-650b-4558-a996-6e7368c0ec72", - "resource": { - "resourceType": "Procedure", - "id": "9f541dc0-650b-4558-a996-6e7368c0ec72", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "169230002", - "display": "Ultrasound scan for fetal viability" - } - ], - "text": "Ultrasound scan for fetal viability" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" - }, - "performedPeriod": { - "start": "2019-08-13T16:24:16-07:00", - "end": "2019-08-13T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:80d606df-cf79-487f-8e24-ca7b42363cee", - "resource": { - "resourceType": "Claim", - "id": "80d606df-cf79-487f-8e24-ca7b42363cee", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-13T16:24:16-07:00", - "end": "2019-08-13T17:09:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "diagnosis": [ - { - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2" - } - } - ], - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:4d516bf3-2917-46e6-87be-51264b421e30" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:9f541dc0-650b-4558-a996-6e7368c0ec72" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:ae6e038e-9e72-45a4-ab30-6f0a24fbd416" - } - ] - }, - { - "sequence": 2, - "diagnosisLinkId": [ - 1 - ] - }, - { - "sequence": 3, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "252160004" - } - ] - }, - "net": { - "value": 4119.91, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "procedureLinkId": [ - 2 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "169230002" - } - ] - }, - "net": { - "value": 14013.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:4f7a5693-ac45-4395-9af8-1c63157fec83", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "4f7a5693-ac45-4395-9af8-1c63157fec83", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "80d606df-cf79-487f-8e24-ca7b42363cee" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-08-13T17:09:16-07:00", - "end": "2020-08-13T17:09:16-07:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "diagnosis": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-inpatient-clm-poa-ind-sw1-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-poa-ind-sw1", - "code": "Y", - "display": "Diagnosis present at time of admission" - } - } - ], - "sequence": 1, - "diagnosisReference": { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2" - }, - "type": [ - { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/diagnosis-type", - "code": "principal" - } - ] - } - ] - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "252160004" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 823.982, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 3295.928, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 4119.91, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 4119.91, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "169230002" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2802.7000000000003, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 11210.800000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 14013.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 14013.50, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 14506.728000000001, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025", - "resource": { - "resourceType": "Encounter", - "id": "3c298587-9c8d-49c3-bde7-7d1e77e12025", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "270427003", - "display": "Patient-initiated encounter" - } - ], - "text": "Patient-initiated encounter" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-08-20T16:24:16-07:00", - "end": "2019-08-20T17:09:16-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04", - "resource": { - "resourceType": "Procedure", - "id": "1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "10383002", - "display": "Counseling for termination of pregnancy" - } - ], - "text": "Counseling for termination of pregnancy" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" - }, - "performedPeriod": { - "start": "2019-08-20T16:24:16-07:00", - "end": "2019-08-20T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:60a85be9-d4a5-42c3-91ed-ee279f501f4a", - "resource": { - "resourceType": "Procedure", - "id": "60a85be9-d4a5-42c3-91ed-ee279f501f4a", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "714812005", - "display": "Induced termination of pregnancy" - } - ], - "text": "Induced termination of pregnancy" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" - }, - "performedPeriod": { - "start": "2019-08-20T16:24:16-07:00", - "end": "2019-08-20T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:edb4ac5a-d52f-4339-85f0-36719a130c3f", - "resource": { - "resourceType": "Claim", - "id": "edb4ac5a-d52f-4339-85f0-36719a130c3f", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-20T16:24:16-07:00", - "end": "2019-08-20T17:09:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:1a20b1ab-50f9-4aab-9e78-9c93e8fb1b04" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:60a85be9-d4a5-42c3-91ed-ee279f501f4a" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:3c298587-9c8d-49c3-bde7-7d1e77e12025" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "10383002" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 2 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "714812005" - } - ] - }, - "net": { - "value": 11765.34, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:31eb2daf-cb8f-4278-bf01-a9d1e67efed7", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "31eb2daf-cb8f-4278-bf01-a9d1e67efed7", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "edb4ac5a-d52f-4339-85f0-36719a130c3f" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-08-20T17:09:16-07:00", - "end": "2020-08-20T17:09:16-07:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "10383002" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "714812005" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 2353.068, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 9412.272, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 11765.34, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 11765.34, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 9825.592, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03", - "resource": { - "resourceType": "Encounter", - "id": "a7de75eb-43a6-4482-957f-4e9c2b82aa03", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "424619006", - "display": "Prenatal visit" - } - ], - "text": "Prenatal visit" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T17:24:16-07:00" - }, - "reason": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "72892002", - "display": "Normal pregnancy" - } - ] - } - ], - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:0669467d-c7a0-478e-a556-1bdc1674a65c", - "resource": { - "resourceType": "Procedure", - "id": "0669467d-c7a0-478e-a556-1bdc1674a65c", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "386394001", - "display": "Pregnancy termination care" - } - ], - "text": "Pregnancy termination care" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" - }, - "performedPeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:5b87d1f8-e02f-4d3e-9f8d-faf9c4417962", - "resource": { - "resourceType": "Procedure", - "id": "5b87d1f8-e02f-4d3e-9f8d-faf9c4417962", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "5880005", - "display": "Physical exam following abortion" - } - ], - "text": "Physical exam following abortion" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" - }, - "performedPeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:8a47b48c-e479-4d1c-b048-477f419db4f3", - "resource": { - "resourceType": "Procedure", - "id": "8a47b48c-e479-4d1c-b048-477f419db4f3", - "status": "completed", - "code": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "171207006", - "display": "Depression screening" - } - ], - "text": "Depression screening" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" - }, - "performedPeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "reasonReference": [ - { - "reference": "urn:uuid:c9731186-6731-4a66-82e2-0669427caeb2", - "display": "Normal pregnancy" - } - ] - }, - "request": { - "method": "POST", - "url": "Procedure" - } - }, - { - "fullUrl": "urn:uuid:30af96fa-867b-4217-9413-fda59bf8f161", - "resource": { - "resourceType": "Claim", - "id": "30af96fa-867b-4217-9413-fda59bf8f161", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T17:24:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "procedure": [ - { - "sequence": 1, - "procedureReference": { - "reference": "urn:uuid:0669467d-c7a0-478e-a556-1bdc1674a65c" - } - }, - { - "sequence": 2, - "procedureReference": { - "reference": "urn:uuid:5b87d1f8-e02f-4d3e-9f8d-faf9c4417962" - } - }, - { - "sequence": 3, - "procedureReference": { - "reference": "urn:uuid:8a47b48c-e479-4d1c-b048-477f419db4f3" - } - } - ], - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:a7de75eb-43a6-4482-957f-4e9c2b82aa03" - } - ] - }, - { - "sequence": 2, - "procedureLinkId": [ - 1 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "386394001" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 3, - "procedureLinkId": [ - 2 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "5880005" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "sequence": 4, - "procedureLinkId": [ - 3 - ], - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "171207006" - } - ] - }, - "net": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:7eeab9d4-aba7-4271-99de-4af8827684bd", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "7eeab9d4-aba7-4271-99de-4af8827684bd", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "30af96fa-867b-4217-9413-fda59bf8f161" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-08-27T17:24:16-07:00", - "end": "2020-08-27T17:24:16-07:00" - }, - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - } - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 2, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "386394001" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 3, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "5880005" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - }, - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 4, - "service": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ide-ndc-upc-num-extension", - "valueCoding": { - "system": "https://www.accessdata.fda.gov/scripts/cder/ndc", - "code": "0624", - "display": "Dummy" - } - } - ], - "coding": [ - { - "system": "http://snomed.info/sct", - "version": "v1", - "code": "171207006" - } - ] - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "adjudication": [ - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_coinsrnc_amt", - "display": "Line Beneficiary Coinsurance Amount" - } - ] - }, - "amount": { - "value": 103.33, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prvdr_pmt_amt", - "display": "Line Provider Payment Amount" - } - ] - }, - "amount": { - "value": 413.32, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_sbmtd_chrg_amt", - "display": "Line Submitted Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_alowd_chrg_amt", - "display": "Line Allowed Charge Amount" - } - ] - }, - "amount": { - "value": 516.65, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_bene_ptb_ddctbl_amt", - "display": "Line Beneficiary Part B Deductible Amount" - } - ] - }, - "amount": { - "value": 0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/codesystem/adjudication", - "code": "https://bluebutton.cms.gov/resources/variables/line_prcsg_ind_cd", - "display": "Line Processing Indicator Code" - } - ] - } - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 1239.96, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155", - "resource": { - "resourceType": "Encounter", - "id": "51eef314-d8e7-4403-abb5-b8072ed4c155", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "698314001", - "display": "Consultation for treatment" - } - ], - "text": "Consultation for treatment" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - } - ], - "period": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "serviceProvider": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:6b4ab930-b750-4f10-a0e1-df0c554105a9", - "resource": { - "resourceType": "MedicationRequest", - "id": "6b4ab930-b750-4f10-a0e1-df0c554105a9", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "748879", - "display": "Levora 0.15/30 28 Day Pack" - } - ], - "text": "Levora 0.15/30 28 Day Pack" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" - }, - "authoredOn": "2019-08-27T16:24:16-07:00", - "requester": { - "agent": { - "reference": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9" - }, - "onBehalfOf": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - } - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:fb9d7e1e-9e63-48ba-9953-a4baab04127e", - "resource": { - "resourceType": "Claim", - "id": "fb9d7e1e-9e63-48ba-9953-a4baab04127e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "prescription": { - "reference": "urn:uuid:6b4ab930-b750-4f10-a0e1-df0c554105a9" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" - } - ] - } - ], - "total": { - "value": 29.37, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:20b108b0-e536-4925-b510-239dc772019e", - "resource": { - "resourceType": "Claim", - "id": "20b108b0-e536-4925-b510-239dc772019e", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "organization": { - "reference": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:255419e8-802f-4fb1-8fd2-a016e9b98a67", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "255419e8-802f-4fb1-8fd2-a016e9b98a67", - "contained": [ - { - "resourceType": "ReferralRequest", - "id": "1", - "status": "completed", - "intent": "order", - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "requester": { - "agent": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - }, - "recipient": [ - { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - } - ] - }, - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "20b108b0-e536-4925-b510-239dc772019e" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-08-27T16:39:16-07:00", - "end": "2020-08-27T16:39:16-07:00" - }, - "created": "2019-08-27T16:39:16-07:00", - "provider": { - "identifier": { - "value": "42f9935b-696e-37c9-8261-2a8c72d02dc9" - } - }, - "organization": { - "identifier": { - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - }, - "referral": { - "reference": "#1" - }, - "claim": { - "reference": "urn:uuid:20b108b0-e536-4925-b510-239dc772019e" - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "sequence": 1, - "category": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_cms_type_srvc_cd", - "code": "1", - "display": "Medical care" - } - ] - }, - "servicedPeriod": { - "start": "2019-08-27T16:24:16-07:00", - "end": "2019-08-27T16:39:16-07:00" - }, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "21", - "display": "Inpatient Hospital" - } - ] - }, - "encounter": [ - { - "reference": "urn:uuid:51eef314-d8e7-4403-abb5-b8072ed4c155" - } - ] - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - }, - { - "fullUrl": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b", - "resource": { - "resourceType": "Encounter", - "id": "5c33c120-05b1-44c0-9156-ca9b8610d22b", - "status": "finished", - "class": { - "system": "http://terminology.hl7.org/CodeSystem/v3-ActCode", - "code": "AMB" - }, - "type": [ - { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "162673000", - "display": "General examination of patient (procedure)" - } - ], - "text": "General examination of patient (procedure)" - } - ], - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "participant": [ - { - "individual": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - } - } - ], - "period": { - "start": "2019-12-10T15:24:16-08:00", - "end": "2019-12-10T15:39:16-08:00" - }, - "serviceProvider": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "request": { - "method": "POST", - "url": "Encounter" - } - }, - { - "fullUrl": "urn:uuid:5e921122-3124-49bd-adec-481c0822393c", - "resource": { - "resourceType": "Observation", - "id": "5e921122-3124-49bd-adec-481c0822393c", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8302-2", - "display": "Body Height" - } - ], - "text": "Body Height" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 160.5, - "unit": "cm", - "system": "http://unitsofmeasure.org", - "code": "cm" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:bb03f471-0162-491c-bba0-f92a4427b6b9", - "resource": { - "resourceType": "Observation", - "id": "bb03f471-0162-491c-bba0-f92a4427b6b9", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72514-3", - "display": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - } - ], - "text": "Pain severity - 0-10 verbal numeric rating [Score] - Reported" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 2, - "unit": "{score}", - "system": "http://unitsofmeasure.org", - "code": "{score}" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:9c4305d5-c593-47b2-8018-a433a33ca0e1", - "resource": { - "resourceType": "Observation", - "id": "9c4305d5-c593-47b2-8018-a433a33ca0e1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "29463-7", - "display": "Body Weight" - } - ], - "text": "Body Weight" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 57.100, - "unit": "kg", - "system": "http://unitsofmeasure.org", - "code": "kg" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b1a92a81-99a1-4a1e-8d61-ff1d9873f5e7", - "resource": { - "resourceType": "Observation", - "id": "b1a92a81-99a1-4a1e-8d61-ff1d9873f5e7", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "39156-5", - "display": "Body Mass Index" - } - ], - "text": "Body Mass Index" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 22.170, - "unit": "kg/m2", - "system": "http://unitsofmeasure.org", - "code": "kg/m2" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:48d08279-c0f6-45fe-a213-fcf384be3dc2", - "resource": { - "resourceType": "Observation", - "id": "48d08279-c0f6-45fe-a213-fcf384be3dc2", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "85354-9", - "display": "Blood Pressure" - } - ], - "text": "Blood Pressure" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "component": [ - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8462-4", - "display": "Diastolic Blood Pressure" - } - ], - "text": "Diastolic Blood Pressure" - }, - "valueQuantity": { - "value": 72, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - }, - { - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8480-6", - "display": "Systolic Blood Pressure" - } - ], - "text": "Systolic Blood Pressure" - }, - "valueQuantity": { - "value": 110, - "unit": "mm[Hg]", - "system": "http://unitsofmeasure.org", - "code": "mm[Hg]" - } - } - ] - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:5301d95a-69cc-42d3-8356-afad95c275d1", - "resource": { - "resourceType": "Observation", - "id": "5301d95a-69cc-42d3-8356-afad95c275d1", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "8867-4", - "display": "Heart rate" - } - ], - "text": "Heart rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 91, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:99cedcfc-fff6-4203-9809-7e2587d1f226", - "resource": { - "resourceType": "Observation", - "id": "99cedcfc-fff6-4203-9809-7e2587d1f226", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "vital-signs", - "display": "vital-signs" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "9279-1", - "display": "Respiratory rate" - } - ], - "text": "Respiratory rate" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueQuantity": { - "value": 14, - "unit": "/min", - "system": "http://unitsofmeasure.org", - "code": "/min" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:56f59c4e-1b31-4c26-96fb-258939b6f57e", - "resource": { - "resourceType": "Observation", - "id": "56f59c4e-1b31-4c26-96fb-258939b6f57e", - "status": "final", - "category": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/observation-category", - "code": "survey", - "display": "survey" - } - ] - } - ], - "code": { - "coding": [ - { - "system": "http://loinc.org", - "code": "72166-2", - "display": "Tobacco smoking status NHIS" - } - ], - "text": "Tobacco smoking status NHIS" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "effectiveDateTime": "2019-12-10T15:24:16-08:00", - "issued": "2019-12-10T15:24:16.644-08:00", - "valueCodeableConcept": { - "coding": [ - { - "system": "http://snomed.info/sct", - "code": "266919005", - "display": "Never smoker" - } - ], - "text": "Never smoker" - } - }, - "request": { - "method": "POST", - "url": "Observation" - } - }, - { - "fullUrl": "urn:uuid:b01481fb-444c-4da9-a31a-d6bdbf57b34a", - "resource": { - "resourceType": "MedicationRequest", - "id": "b01481fb-444c-4da9-a31a-d6bdbf57b34a", - "status": "active", - "intent": "order", - "medicationCodeableConcept": { - "coding": [ - { - "system": "http://www.nlm.nih.gov/research/umls/rxnorm", - "code": "746030", - "display": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - } - ], - "text": "Atenolol 50 MG / Chlorthalidone 25 MG Oral Tablet" - }, - "subject": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "context": { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - }, - "authoredOn": "2019-12-10T15:24:16-08:00", - "requester": { - "agent": { - "reference": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915" - }, - "onBehalfOf": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "reasonReference": [ - { - "reference": "urn:uuid:42372dca-1642-40a5-acfa-8ecddbf32701" - } - ] - }, - "request": { - "method": "POST", - "url": "MedicationRequest" - } - }, - { - "fullUrl": "urn:uuid:39525a20-1c93-4991-88ee-657bb411d983", - "resource": { - "resourceType": "Claim", - "id": "39525a20-1c93-4991-88ee-657bb411d983", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-12-10T15:24:16-08:00", - "end": "2019-12-10T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "prescription": { - "reference": "urn:uuid:b01481fb-444c-4da9-a31a-d6bdbf57b34a" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - } - ] - } - ], - "total": { - "value": 263.49, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:65dda29e-1896-4adb-8c7d-c7d45a1590f3", - "resource": { - "resourceType": "Claim", - "id": "65dda29e-1896-4adb-8c7d-c7d45a1590f3", - "status": "active", - "use": "complete", - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "start": "2019-12-10T15:24:16-08:00", - "end": "2019-12-10T15:39:16-08:00" - }, - "organization": { - "reference": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4" - }, - "item": [ - { - "sequence": 1, - "encounter": [ - { - "reference": "urn:uuid:5c33c120-05b1-44c0-9156-ca9b8610d22b" - } - ] - } - ], - "total": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - "request": { - "method": "POST", - "url": "Claim" - } - }, - { - "fullUrl": "urn:uuid:5ae92d6e-1b87-4356-8957-93d63a82e409", - "resource": { - "resourceType": "ExplanationOfBenefit", - "id": "5ae92d6e-1b87-4356-8957-93d63a82e409", - "meta": { - "profile": [ - "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim" - ] - }, - "contained": [ - { - "resourceType": "Coverage", - "id": "coverage", - "type": { - "text": "Blue Cross Blue Shield" - } - } - ], - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-profnl-cmpnt-chrg-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-ddctbl-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-ptb-coinsrnc-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-prvdr-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-op-bene-pmt-amt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-nch-bene-blood-ddctbl-lblty-am-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-clm-mdcr-non-pmt-rsn-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-clm-mdcr-non-pmt-rsn-cd", - "code": "N", - "display": "All other reasons for non-payment" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-prpayamt-extension", - "valueMoney": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - }, - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-fi-num-extension", - "valueIdentifier": { - "system": "https://bluebutton.cms.gov/assets/ig/CodeSystem-fi-num", - "value": "002000" - } - } - ], - "identifier": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/clm_id", - "value": "65dda29e-1896-4adb-8c7d-c7d45a1590f3" - }, - { - "system": "https://bluebutton.cms.gov/resources/identifier/claim-group", - "value": "99999999999" - } - ], - "status": "active", - "type": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_clm_type_cd", - "code": "71", - "display": "Local carrier non-durable medical equipment, prosthetics, orthotics, and supplies (DMEPOS) claim" - }, - { - "system": "https://bluebutton.cms.gov/resources/codesystem/eob-type", - "code": "CARRIER", - "display": "EOB Type" - }, - { - "system": "http://hl7.org/fhir/ex-claimtype", - "code": "professional", - "display": "Claim Type" - }, - { - "system": "https://bluebutton.cms.gov/resources/variables/nch_near_line_rec_ident_cd", - "code": "O", - "display": "Part B physician/supplier claim record (processed by local carriers; can include DMEPOS services)" - } - ] - }, - "patient": { - "reference": "urn:uuid:3024090f-fe14-40a9-8fae-79952d3c95ce" - }, - "billablePeriod": { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-claim-query-cd-extension", - "valueCoding": { - "system": "https://bluebutton.cms.gov/assets/ig/ValueSet-claim-query-cd", - "code": "3", - "display": "Final Bill" - } - } - ], - "start": "2019-12-10T15:39:16-08:00", - "end": "2020-12-10T15:39:16-08:00" - }, - "provider": { - "identifier": { - "value": "7212c355-6c66-3c09-b527-bb8d33b0a915" - } - }, - "organization": { - "identifier": { - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - }, - "careTeam": [ - { - "sequence": 1, - "provider": { - "identifier": { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "99999999" - } - }, - "role": { - "coding": [ - { - "system": "http://hl7.org/fhir/claimcareteamrole", - "code": "primary", - "display": "Primary Care Practitioner" - } - ] - } - } - ], - "insurance": { - "coverage": { - "reference": "#coverage" - } - }, - "item": [ - { - "extension": [ - { - "url": "https://bluebutton.cms.gov/assets/ig/StructureDefinition-bluebutton-outpatient-rev-cntr-ndc-qty-extension", - "valueQuantity": { - "value": 0 - } - } - ], - "sequence": 1, - "locationCodeableConcept": { - "coding": [ - { - "system": "https://bluebutton.cms.gov/resources/variables/line_place_of_srvc_cd", - "code": "22", - "display": "Outpatient Hospital" - } - ] - } - } - ], - "totalCost": { - "value": 129.16, - "system": "urn:iso:std:iso:4217", - "code": "USD" - }, - "payment": { - "amount": { - "value": 0.0, - "system": "urn:iso:std:iso:4217", - "code": "USD" - } - } - }, - "request": { - "method": "POST", - "url": "ExplanationOfBenefit" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json deleted file mode 100644 index 02eefcfa77eb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/hospitalInformation1586298239556.json +++ /dev/null @@ -1,880 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:69176529-fd1f-3b3f-abce-a0a3626769eb", - "resource": { - "resourceType": "Organization", - "id": "69176529-fd1f-3b3f-abce-a0a3626769eb", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 11 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "69176529-fd1f-3b3f-abce-a0a3626769eb" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "MOUNT AUBURN HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "6174923500" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:e002090d-4e92-300e-b41e-7d1f21dee4c6", - "resource": { - "resourceType": "Organization", - "id": "e002090d-4e92-300e-b41e-7d1f21dee4c6", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 9 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "e002090d-4e92-300e-b41e-7d1f21dee4c6" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "CAMBRIDGE HEALTH ALLIANCE", - "telecom": [ - { - "system": "phone", - "value": "6176652300" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.375967 - }, - { - "url": "longitude", - "valueDecimal": -71.118275 - } - ] - } - ], - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:5844ad77-f653-3c2b-b7dd-e97576ab3b03", - "resource": { - "resourceType": "Organization", - "id": "5844ad77-f653-3c2b-b7dd-e97576ab3b03", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 46 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "5844ad77-f653-3c2b-b7dd-e97576ab3b03" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "BAYSTATE WING HOSPITAL AND MEDICAL CENTERS", - "telecom": [ - { - "system": "phone", - "value": "4132837651" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.187794 - }, - { - "url": "longitude", - "valueDecimal": -72.30846899999997 - } - ] - } - ], - "line": [ - "40 WRIGHT STREET" - ], - "city": "PALMER", - "state": "MA", - "postalCode": "01069", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:c44f361c-2efb-3050-8f97-0354a12e2920", - "resource": { - "resourceType": "Organization", - "id": "c44f361c-2efb-3050-8f97-0354a12e2920", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "c44f361c-2efb-3050-8f97-0354a12e2920" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SIGNATURE HEALTHCARE BROCKTON HOSPITAL", - "telecom": [ - { - "system": "phone", - "value": "5089417000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:226098a2-6a40-3588-b5bb-db56c3a30a04", - "resource": { - "resourceType": "Organization", - "id": "226098a2-6a40-3588-b5bb-db56c3a30a04", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 8 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "226098a2-6a40-3588-b5bb-db56c3a30a04" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "GOOD SAMARITAN MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5084273000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.082543 - }, - { - "url": "longitude", - "valueDecimal": -71.024638 - } - ] - } - ], - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:9bd6ea26-d344-3409-aea7-2e925fd0d245", - "resource": { - "resourceType": "Organization", - "id": "9bd6ea26-d344-3409-aea7-2e925fd0d245", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 17 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 9 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 8 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "9bd6ea26-d344-3409-aea7-2e925fd0d245" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP32892", - "telecom": [ - { - "system": "phone", - "value": "617-744-6527" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.369451 - }, - { - "url": "longitude", - "valueDecimal": -71.177925 - } - ] - } - ], - "line": [ - "158 A BELMONT ST" - ], - "city": "WATERTOWN", - "state": "MA", - "postalCode": "02472-2912", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:97066a41-4fd5-3d4e-b382-a978fb20a5d5", - "resource": { - "resourceType": "Organization", - "id": "97066a41-4fd5-3d4e-b382-a978fb20a5d5", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 15 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 11 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 1 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "97066a41-4fd5-3d4e-b382-a978fb20a5d5" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP33234", - "telecom": [ - { - "system": "phone", - "value": "781-829-9300" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.122956 - }, - { - "url": "longitude", - "valueDecimal": -70.85631 - } - ] - } - ], - "line": [ - "198 COLUMBIA RD" - ], - "city": "HANOVER", - "state": "MA", - "postalCode": "02339-2380", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:aced4ce4-957a-3d86-b20a-302d12f949a5", - "resource": { - "resourceType": "Organization", - "id": "aced4ce4-957a-3d86-b20a-302d12f949a5", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 9 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 7 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 5 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "aced4ce4-957a-3d86-b20a-302d12f949a5" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "LONGFELLOW PRIMARY CARE, PC", - "telecom": [ - { - "system": "phone", - "value": "617-492-4545" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.376043 - }, - { - "url": "longitude", - "valueDecimal": -71.11868 - } - ] - } - ], - "line": [ - "625 MOUNT AUBURN ST" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138-4518", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:83284578-12e5-3582-bfe1-7807e3f7a212", - "resource": { - "resourceType": "Organization", - "id": "83284578-12e5-3582-bfe1-7807e3f7a212", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 8 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 8 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 1 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 4 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "83284578-12e5-3582-bfe1-7807e3f7a212" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP297713", - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.423844 - }, - { - "url": "longitude", - "valueDecimal": -71.10923100000001 - } - ] - } - ], - "line": [ - "0 GOVERNORS AVE" - ], - "city": "MEDFORD", - "state": "MA", - "postalCode": "02155-3084", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", - "resource": { - "resourceType": "Organization", - "id": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 20 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 9 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 3 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 4 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "7bdd7bd6-d521-3b19-81b6-ce2ed80d9e74" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "SIGNATURE HEALTHCARE MEDICAL GROUP INC", - "telecom": [ - { - "system": "phone", - "value": "781-878-1700" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.119966999999995 - }, - { - "url": "longitude", - "valueDecimal": -70.957211 - } - ] - } - ], - "line": [ - "360 BROCKTON AVE" - ], - "city": "ABINGTON", - "state": "MA", - "postalCode": "02351-2186", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - }, - { - "fullUrl": "urn:uuid:883860d6-0862-3a47-a18f-7256563c53f4", - "resource": { - "resourceType": "Organization", - "id": "883860d6-0862-3a47-a18f-7256563c53f4", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 34 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 50 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 5 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 29 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "883860d6-0862-3a47-a18f-7256563c53f4" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "HARRINGTON PHYSICIAN SERVICES INC", - "telecom": [ - { - "system": "phone", - "value": "413-245-0966" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.128176 - }, - { - "url": "longitude", - "valueDecimal": -72.205352 - } - ] - } - ], - "line": [ - "255 E OLD STURBRIDGE RD" - ], - "city": "BRIMFIELD", - "state": "MA", - "postalCode": "01010-9647", - "country": "US" - } - ] - }, - "request": { - "method": "POST", - "url": "Organization" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json b/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json deleted file mode 100644 index 32853f75bf0c..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/STU3/practitionerInformation1586298239556.json +++ /dev/null @@ -1,523 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "transaction", - "entry": [ - { - "fullUrl": "urn:uuid:339a8b0f-de8f-3168-bfe3-89f8b4614840", - "resource": { - "resourceType": "Practitioner", - "id": "339a8b0f-de8f-3168-bfe3-89f8b4614840", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 11 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "10" - } - ], - "active": true, - "name": [ - { - "family": "Jenkins714", - "given": [ - "Lara964" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "330 MOUNT AUBURN STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:6a3782fa-7d6e-302f-bec5-695563b123a8", - "resource": { - "resourceType": "Practitioner", - "id": "6a3782fa-7d6e-302f-bec5-695563b123a8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 9 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "40" - } - ], - "active": true, - "name": [ - { - "family": "Gibson10", - "given": [ - "Loretta235" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "1493 CAMBRIDGE STREET" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:42f9935b-696e-37c9-8261-2a8c72d02dc9", - "resource": { - "resourceType": "Practitioner", - "id": "42f9935b-696e-37c9-8261-2a8c72d02dc9", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 46 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "130" - } - ], - "active": true, - "name": [ - { - "family": "Abernathy524", - "given": [ - "Kirby843" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "40 WRIGHT STREET" - ], - "city": "PALMER", - "state": "MA", - "postalCode": "01069", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:d08d5908-40a4-3571-8c54-0cc029c8277f", - "resource": { - "resourceType": "Practitioner", - "id": "d08d5908-40a4-3571-8c54-0cc029c8277f", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "200" - } - ], - "active": true, - "name": [ - { - "family": "Padberg411", - "given": [ - "Nila48" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "680 CENTER STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02302", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "resource": { - "resourceType": "Practitioner", - "id": "27fac077-3105-3983-8b0f-cc4f30f9e7c1", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 8 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "460" - } - ], - "active": true, - "name": [ - { - "family": "Fay398", - "given": [ - "Deneen201" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "235 NORTH PEARL STREET" - ], - "city": "BROCKTON", - "state": "MA", - "postalCode": "02301", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:ba314750-701e-3370-ade0-270e178d80e0", - "resource": { - "resourceType": "Practitioner", - "id": "ba314750-701e-3370-ade0-270e178d80e0", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 17 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "22240" - } - ], - "active": true, - "name": [ - { - "family": "Pacocha935", - "given": [ - "Ula130" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "158 A BELMONT ST" - ], - "city": "WATERTOWN", - "state": "MA", - "postalCode": "02472-2912", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:f9f09686-21e4-3b27-ba68-9b7313d9a553", - "resource": { - "resourceType": "Practitioner", - "id": "f9f09686-21e4-3b27-ba68-9b7313d9a553", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 15 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "22570" - } - ], - "active": true, - "name": [ - { - "family": "Homenick806", - "given": [ - "Evan94" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "198 COLUMBIA RD" - ], - "city": "HANOVER", - "state": "MA", - "postalCode": "02339-2380", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:29e6e246-9052-309c-a47f-c2e5bc60c886", - "resource": { - "resourceType": "Practitioner", - "id": "29e6e246-9052-309c-a47f-c2e5bc60c886", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 9 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "24070" - } - ], - "active": true, - "name": [ - { - "family": "Waelchi213", - "given": [ - "Cristopher265" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "625 MOUNT AUBURN ST" - ], - "city": "CAMBRIDGE", - "state": "MA", - "postalCode": "02138-4518", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:d726e757-2975-3d6a-bdf9-517546432499", - "resource": { - "resourceType": "Practitioner", - "id": "d726e757-2975-3d6a-bdf9-517546432499", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 8 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "80690" - } - ], - "active": true, - "name": [ - { - "family": "Abernathy524", - "given": [ - "Ulrike696" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "0 GOVERNORS AVE" - ], - "city": "MEDFORD", - "state": "MA", - "postalCode": "02155-3084", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:3dde31b6-dbc2-394b-b2af-a3167d1f6841", - "resource": { - "resourceType": "Practitioner", - "id": "3dde31b6-dbc2-394b-b2af-a3167d1f6841", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 20 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "81770" - } - ], - "active": true, - "name": [ - { - "family": "Ondricka197", - "given": [ - "Leisa54" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "360 BROCKTON AVE" - ], - "city": "ABINGTON", - "state": "MA", - "postalCode": "02351-2186", - "country": "US" - } - ], - "gender": "female" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - }, - { - "fullUrl": "urn:uuid:7212c355-6c66-3c09-b527-bb8d33b0a915", - "resource": { - "resourceType": "Practitioner", - "id": "7212c355-6c66-3c09-b527-bb8d33b0a915", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 34 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "85760" - } - ], - "active": true, - "name": [ - { - "family": "Kihn564", - "given": [ - "Hayden835" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "255 E OLD STURBRIDGE RD" - ], - "city": "BRIMFIELD", - "state": "MA", - "postalCode": "01010-9647", - "country": "US" - } - ], - "gender": "male" - }, - "request": { - "method": "POST", - "url": "Practitioner" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json deleted file mode 100644 index 9cf9888c287a..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/hospitalInformation1589831190233.json +++ /dev/null @@ -1,183 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", - "resource": { - "resourceType": "Organization", - "id": "465de31f-3098-365c-af70-48a071e1f5aa", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "465de31f-3098-365c-af70-48a071e1f5aa" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "METROWEST MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5083831000" - } - ], - "address": [ - { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "resource": { - "resourceType": "Organization", - "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 23 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 5 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 17 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "PCP68975", - "telecom": [ - { - "system": "phone", - "value": "508-881-4368" - } - ], - "address": [ - { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "resource": { - "resourceType": "Organization", - "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" - } - ], - "type": { - "coding": [ - { - "system": "Healthcare Provider", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - }, - "name": "WALTHAM URGENT CARE", - "telecom": [ - { - "system": "phone", - "value": "617-243-5591" - } - ], - "address": [ - { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ] - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json deleted file mode 100644 index 76f865f73c7c..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/DSTU2/practitionerInformation1589831190233.json +++ /dev/null @@ -1,135 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", - "resource": { - "resourceType": "Practitioner", - "id": "c16820ae-2954-32d4-863c-e9ceb741154c", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "530" - } - ], - "active": true, - "name": { - "family": [ - "Murphy561" - ], - "given": [ - "Mari763" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "resource": { - "resourceType": "Practitioner", - "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "35750" - } - ], - "active": true, - "name": { - "family": [ - "Hilpert278" - ], - "given": [ - "Cathryn51" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", - "resource": { - "resourceType": "Practitioner", - "id": "861aa935-d89f-3b41-861b-b791fa112253", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93440" - } - ], - "active": true, - "name": { - "family": [ - "Reynolds644" - ], - "given": [ - "Salena230" - ], - "prefix": [ - "Dr." - ] - }, - "address": [ - { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ], - "gender": "female" - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json deleted file mode 100644 index b80501554861..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/hospitalInformation1589831190233.json +++ /dev/null @@ -1,318 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:5cd31fbf-76ba-4e8c-84dd-6c72071c98cf", - "resource": { - "resourceType": "Location", - "id": "5cd31fbf-76ba-4e8c-84dd-6c72071c98cf", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" - ] - }, - "status": "active", - "name": "METROWEST MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5083831000" - } - ], - "address": { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - }, - "position": { - "longitude": -71.436196, - "latitude": 42.307905 - }, - "managingOrganization": { - "reference": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", - "display": "METROWEST MEDICAL CENTER" - } - } - }, - { - "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", - "resource": { - "resourceType": "Organization", - "id": "465de31f-3098-365c-af70-48a071e1f5aa", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "465de31f-3098-365c-af70-48a071e1f5aa" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "METROWEST MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5083831000" - } - ], - "address": [ - { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:274697c5-2d9d-412f-b825-194f88241aec", - "resource": { - "resourceType": "Location", - "id": "274697c5-2d9d-412f-b825-194f88241aec", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" - ] - }, - "status": "active", - "name": "PCP68975", - "telecom": [ - { - "system": "phone", - "value": "508-881-4368" - } - ], - "address": { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - }, - "position": { - "longitude": -71.473526, - "latitude": 42.257754999999996 - }, - "managingOrganization": { - "reference": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "display": "PCP68975" - } - } - }, - { - "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "resource": { - "resourceType": "Organization", - "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 23 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 5 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 17 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP68975", - "telecom": [ - { - "system": "phone", - "value": "508-881-4368" - } - ], - "address": [ - { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:31352f93-39d4-447d-bb77-8cac27d5c9d0", - "resource": { - "resourceType": "Location", - "id": "31352f93-39d4-447d-bb77-8cac27d5c9d0", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-location" - ] - }, - "status": "active", - "name": "WALTHAM URGENT CARE", - "telecom": [ - { - "system": "phone", - "value": "617-243-5591" - } - ], - "address": { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - }, - "position": { - "longitude": -71.2490152, - "latitude": 42.3700513 - }, - "managingOrganization": { - "reference": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "display": "WALTHAM URGENT CARE" - } - } - }, - { - "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "resource": { - "resourceType": "Organization", - "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-organization" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" - } - ], - "active": true, - "type": [ - { - "coding": [ - { - "system": "http://terminology.hl7.org/CodeSystem/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "WALTHAM URGENT CARE", - "telecom": [ - { - "system": "phone", - "value": "617-243-5591" - } - ], - "address": [ - { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ] - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json deleted file mode 100644 index 90fcfcc493a2..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/R4/practitionerInformation1589831190233.json +++ /dev/null @@ -1,354 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", - "resource": { - "resourceType": "Practitioner", - "id": "c16820ae-2954-32d4-863c-e9ceb741154c", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "530" - } - ], - "active": true, - "name": [ - { - "family": "Murphy561", - "given": [ - "Mari763" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", - "valueBoolean": true - } - ], - "system": "email", - "value": "Mari763.Murphy561@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:70dbe417-06dc-4c57-93ce-84209ac38ff5", - "resource": { - "resourceType": "PractitionerRole", - "id": "70dbe417-06dc-4c57-93ce-84209ac38ff5", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" - ] - }, - "practitioner": { - "reference": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", - "display": "Dr. Mari763 Murphy561" - }, - "organization": { - "reference": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", - "display": "METROWEST MEDICAL CENTER" - }, - "code": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "specialty": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "location": [ - { - "display": "METROWEST MEDICAL CENTER" - } - ], - "telecom": [ - { - "system": "phone", - "value": "5083831000" - } - ] - } - }, - { - "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "resource": { - "resourceType": "Practitioner", - "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "35750" - } - ], - "active": true, - "name": [ - { - "family": "Hilpert278", - "given": [ - "Cathryn51" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", - "valueBoolean": true - } - ], - "system": "email", - "value": "Cathryn51.Hilpert278@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:d163bdcf-8294-4320-85b8-86e0680dc34a", - "resource": { - "resourceType": "PractitionerRole", - "id": "d163bdcf-8294-4320-85b8-86e0680dc34a", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" - ] - }, - "practitioner": { - "reference": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "display": "Dr. Cathryn51 Hilpert278" - }, - "organization": { - "reference": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "display": "PCP68975" - }, - "code": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "specialty": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "location": [ - { - "display": "PCP68975" - } - ], - "telecom": [ - { - "system": "phone", - "value": "508-881-4368" - } - ] - } - }, - { - "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", - "resource": { - "resourceType": "Practitioner", - "id": "861aa935-d89f-3b41-861b-b791fa112253", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitioner" - ] - }, - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93440" - } - ], - "active": true, - "name": [ - { - "family": "Reynolds644", - "given": [ - "Salena230" - ], - "prefix": [ - "Dr." - ] - } - ], - "telecom": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/us/core/StructureDefinition/us-core-direct", - "valueBoolean": true - } - ], - "system": "email", - "value": "Salena230.Reynolds644@example.com", - "use": "work" - } - ], - "address": [ - { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:689adf0e-16fd-4546-9764-69d26ef64777", - "resource": { - "resourceType": "PractitionerRole", - "id": "689adf0e-16fd-4546-9764-69d26ef64777", - "meta": { - "profile": [ - "http://hl7.org/fhir/us/core/StructureDefinition/us-core-practitionerrole" - ] - }, - "practitioner": { - "reference": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", - "display": "Dr. Salena230 Reynolds644" - }, - "organization": { - "reference": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "display": "WALTHAM URGENT CARE" - }, - "code": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "specialty": [ - { - "coding": [ - { - "system": "http://nucc.org/provider-taxonomy", - "code": "208D00000X", - "display": "General Practice" - } - ], - "text": "General Practice" - } - ], - "location": [ - { - "display": "WALTHAM URGENT CARE" - } - ], - "telecom": [ - { - "system": "phone", - "value": "617-243-5591" - } - ] - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json deleted file mode 100644 index dfe338def8e6..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/hospitalInformation1589831190233.json +++ /dev/null @@ -1,234 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:465de31f-3098-365c-af70-48a071e1f5aa", - "resource": { - "resourceType": "Organization", - "id": "465de31f-3098-365c-af70-48a071e1f5aa", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "465de31f-3098-365c-af70-48a071e1f5aa" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "METROWEST MEDICAL CENTER", - "telecom": [ - { - "system": "phone", - "value": "5083831000" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.307905 - }, - { - "url": "longitude", - "valueDecimal": -71.436196 - } - ] - } - ], - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "resource": { - "resourceType": "Organization", - "id": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 23 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 5 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 17 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "58fe1815-1e8a-38ed-a91a-17d4ef18c8d8" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "PCP68975", - "telecom": [ - { - "system": "phone", - "value": "508-881-4368" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.257754999999996 - }, - { - "url": "longitude", - "valueDecimal": -71.473526 - } - ] - } - ], - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ] - } - }, - { - "fullUrl": "urn:uuid:d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "resource": { - "resourceType": "Organization", - "id": "d672f853-e2a5-324e-98fa-c6d5f8dfc255", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-procedures-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-labs-extension", - "valueInteger": 0 - }, - { - "url": "http://synthetichealth.github.io/synthea/utilization-prescriptions-extension", - "valueInteger": 0 - } - ], - "identifier": [ - { - "system": "https://github.com/synthetichealth/synthea", - "value": "d672f853-e2a5-324e-98fa-c6d5f8dfc255" - } - ], - "type": [ - { - "coding": [ - { - "system": "http://hl7.org/fhir/organization-type", - "code": "prov", - "display": "Healthcare Provider" - } - ], - "text": "Healthcare Provider" - } - ], - "name": "WALTHAM URGENT CARE", - "telecom": [ - { - "system": "phone", - "value": "617-243-5591" - } - ], - "address": [ - { - "extension": [ - { - "url": "http://hl7.org/fhir/StructureDefinition/geolocation", - "extension": [ - { - "url": "latitude", - "valueDecimal": 42.3700513 - }, - { - "url": "longitude", - "valueDecimal": -71.2490152 - } - ] - } - ], - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ] - } - } - ] -} diff --git a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json b/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json deleted file mode 100644 index 9e963d958d03..000000000000 --- a/sdks/java/io/google-cloud-platform/src/test/resources/resources/STU3/practitionerInformation1589831190233.json +++ /dev/null @@ -1,135 +0,0 @@ -{ - "resourceType": "Bundle", - "type": "collection", - "entry": [ - { - "fullUrl": "urn:uuid:c16820ae-2954-32d4-863c-e9ceb741154c", - "resource": { - "resourceType": "Practitioner", - "id": "c16820ae-2954-32d4-863c-e9ceb741154c", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 27 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "530" - } - ], - "active": true, - "name": [ - { - "family": "Murphy561", - "given": [ - "Mari763" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "115 LINCOLN STREET" - ], - "city": "FRAMINGHAM", - "state": "MA", - "postalCode": "01701", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "resource": { - "resourceType": "Practitioner", - "id": "a0f721c3-b8d1-3227-a8bf-27e5bf1ef5e8", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 30 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "35750" - } - ], - "active": true, - "name": [ - { - "family": "Hilpert278", - "given": [ - "Cathryn51" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "259 MAIN ST" - ], - "city": "ASHLAND", - "state": "MA", - "postalCode": "01721-2115", - "country": "US" - } - ], - "gender": "female" - } - }, - { - "fullUrl": "urn:uuid:861aa935-d89f-3b41-861b-b791fa112253", - "resource": { - "resourceType": "Practitioner", - "id": "861aa935-d89f-3b41-861b-b791fa112253", - "extension": [ - { - "url": "http://synthetichealth.github.io/synthea/utilization-encounters-extension", - "valueInteger": 2 - } - ], - "identifier": [ - { - "system": "http://hl7.org/fhir/sid/us-npi", - "value": "93440" - } - ], - "active": true, - "name": [ - { - "family": "Reynolds644", - "given": [ - "Salena230" - ], - "prefix": [ - "Dr." - ] - } - ], - "address": [ - { - "line": [ - "9 HOPE AVENUE" - ], - "city": "WALTHAM", - "state": "MA", - "postalCode": "2453", - "country": "US" - } - ], - "gender": "female" - } - } - ] -} From 39c2d9df0ac1eb592c0fbcc6777a78e75c37e478 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 16:35:02 -0700 Subject: [PATCH 145/151] split create / update ITs --- .../sdk/io/gcp/healthcare/FhirIOCreateIT.java | 119 ++++++++++++++++++ 1 file changed, 119 insertions(+) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java new file mode 100644 index 000000000000..05741bc26d0f --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java @@ -0,0 +1,119 @@ +/* + * 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.beam.sdk.io.gcp.healthcare; + +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.DEFAULT_TEMP_BUCKET; +import static org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.RESOURCES; +import static org.apache.beam.sdk.io.gcp.healthcare.HL7v2IOTestUtil.HEALTHCARE_DATASET_TEMPLATE; + +import java.io.IOException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.KV; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class FhirIOCreateIT { + + @Parameters(name = "{0}") + public static Collection versions() { + return Arrays.asList("DSTU2", "STU3", "R4"); + } + + private final String fhirStoreName; + private FhirIOTestOptions options; + private transient HealthcareApiClient client; + private String healthcareDataset; + private long testTime = System.currentTimeMillis(); + + public String version; + + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + + public FhirIOCreateIT(String version) { + this.version = version; + this.fhirStoreName = + "FHIR_store_" + version + "_write_it_" + testTime + "_" + (new SecureRandom().nextInt(32)); + } + + @Before + public void setup() throws Exception { + if (client == null) { + client = new HttpHealthcareApiClient(); + } + PipelineOptionsFactory.register(FhirIOTestOptions.class); + String project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + healthcareDataset = String.format(HEALTHCARE_DATASET_TEMPLATE, project); + options = TestPipeline.testingPipelineOptions().as(FhirIOTestOptions.class); + options.setGcsTempPath( + String.format("gs://%s/FhirIOWrite%sIT/%s/temp/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setGcsDeadLetterPath( + String.format( + "gs://%s/FhirIOWrite%sIT/%s/deadletter/", DEFAULT_TEMP_BUCKET, version, testTime)); + options.setFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.createFhirStore(healthcareDataset, fhirStoreName, version); + } + + @After + public void teardownFhirStore() throws IOException { + HealthcareApiClient client = new HttpHealthcareApiClient(); + client.deleteFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); + // clean up GCS objects if any. + } + + @AfterClass + public static void teardownBucket() throws IOException { + FhirIOTestUtil.tearDownTempBucket(); + } + + @Test + public void testFhirIO_CreateResources() { + Result createResult = + (Result) + pipeline + .apply("Seed Test Resources", Create.of(RESOURCES.get(version))) + .apply( + "Create FHIR Resources", + FhirIO.createResources(options.getFhirStore()) + .withTypeFunction(new GetByKey("resourceType")) + .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) + .withFormatBodyFunction(x -> x)); + + PAssert.that(createResult.getFailedBodies()).empty(); + + pipeline.run().waitUntilFinish(); + } +} From 3abc1a533378e153a26c7380e511b73989bb9735 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 17:42:29 -0700 Subject: [PATCH 146/151] conditional update IT --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 8 ++-- .../sdk/io/gcp/healthcare/FhirIOCreateIT.java | 6 +-- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 35 ++++++++++++++++ .../sdk/io/gcp/healthcare/FhirIOUpdateIT.java | 40 +++++++------------ 4 files changed, 55 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index c74ec8e3e072..0f07a8a8cc76 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -1538,10 +1538,10 @@ public UpdateResources withETagFunction(SerializableFunction etagFunc public FhirIO.Write.Result expand(PCollection input) { checkArgument( formatBodyFunction != null, - "FhirIO.UpdateResources should always be called with a " + "withFormatBodyFunction."); + "FhirIO.UpdateResources should always be called with a withFormatBodyFunction."); checkArgument( - resourceNameFunction == null, - "resourceNameFunction must be set when using FhirIO.UpdateResources"); + resourceNameFunction != null, + "FhirIO.UpdateResources should always be called with a withResourceNameFunction."); return Write.Result.in( input.getPipeline(), input @@ -1579,7 +1579,7 @@ public void initClient() throws IOException { } @ProcessElement - public void update(ProcessContext context) throws IOException { + public void update(ProcessContext context) { T input = context.element(); String body = formatBodyFunction.apply(input); try { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java index 05741bc26d0f..68cb6fae2d9f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOCreateIT.java @@ -25,16 +25,14 @@ import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.ExtractIDSearchParams; import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.values.KV; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -109,7 +107,7 @@ public void testFhirIO_CreateResources() { "Create FHIR Resources", FhirIO.createResources(options.getFhirStore()) .withTypeFunction(new GetByKey("resourceType")) - .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) + .withIfNotExistFunction(new ExtractIDSearchParams()) .withFormatBodyFunction(x -> x)); PAssert.that(createResult.getFailedBodies()).empty(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 0de5d9062d70..b96d22bef1d0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -66,6 +66,29 @@ public String apply(String resource) { } } + public static class ExtractIDSearchParams + implements SerializableFunction> { + private ObjectMapper mapper; + + ExtractIDSearchParams() { + mapper = new ObjectMapper(); + } + + @Override + public Map apply(String resource) { + Map searchParams = new HashMap<>(); + try { + Map map = + mapper.readValue(resource.getBytes(StandardCharsets.UTF_8), Map.class); + String id = map.get("id"); + searchParams.put("_id", id); + return searchParams; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + public static class GetByKey implements SerializableFunction { private final String key; private ObjectMapper mapper; @@ -159,6 +182,18 @@ static void executeFhirBundles(HealthcareApiClient client, String fhirStore, Lis } } + /** Populate the test resources into the FHIR store and returns a list of resource IDs. */ + static void createFhirResources( + HealthcareApiClient client, String fhirStore, List resources) + throws IOException, HealthcareHttpException { + GetByKey getByKey = new GetByKey("resourceType"); + ExtractIDSearchQuery extractIDSearchQuery = new ExtractIDSearchQuery(); + for (String resource : resources) { + client.fhirCreate( + fhirStore, getByKey.apply(resource), resource, extractIDSearchQuery.apply(resource)); + } + } + public static void tearDownTempBucket() throws IOException { GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java index fe5bad28caf4..629c75b390b4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java @@ -25,10 +25,12 @@ import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; +import java.util.List; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; +import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.ExtractIDSearchParams; import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; +import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -49,7 +51,8 @@ public class FhirIOUpdateIT { @Parameters(name = "{0}") public static Collection versions() { - return Arrays.asList("DSTU2", "STU3", "R4"); + // TODO(jaketf) uncomment other versions. + return Arrays.asList(/*"DSTU2", "STU3", */ "R4"); } private final String fhirStoreName; @@ -100,30 +103,15 @@ public static void teardownBucket() throws IOException { } @Test - public void testFhirIO_CreateResources() { - Result createResult = - (Result) - pipeline - .apply("Seed Test Resources", Create.of(RESOURCES.get(version))) - .apply( - "Create FHIR Resources", - FhirIO.createResources(options.getFhirStore()) - .withTypeFunction(new GetByKey("resourceType")) - .withIfNotExistFunction(new FhirIOTestUtil.ExtractIDSearchQuery()) - .withFormatBodyFunction(x -> x)); - - PAssert.that(createResult.getFailedBodies()).empty(); - - pipeline.run().waitUntilFinish(); - } - - @Test - public void testFhirIO_Update() { + public void testFhirIO_Update() throws IOException, HealthcareHttpException { + List initialResources = RESOURCES.get(version); + FhirIOTestUtil.createFhirResources( + client, healthcareDataset + "/fhirStores/" + fhirStoreName, initialResources); // TODO write initial resources to FHIR Result updateResult = (Result) pipeline - .apply("Seed Test Resources", Create.of(RESOURCES.get(version))) + .apply("Seed Test Resources", Create.of(initialResources)) .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) .apply( "Update Resources", @@ -147,9 +135,9 @@ public void testFhirIO_ConditionalUpdate() { .apply( "Conditional Update Resources", FhirIO.conditionalUpdate(options.getFhirStore()) - .withTypeFunction(x -> "patient") - .withFormatBodyFunction(x -> "{}") - .withSearchParametersFunction(x -> new HashMap<>())); - // TODO spot check update results + .withTypeFunction(new GetByKey("resourceType")) + .withFormatBodyFunction(x -> x) + .withSearchParametersFunction(new ExtractIDSearchParams())); + pipeline.run().waitUntilFinish(); } } From 6e1372cb7ee03933ac8c94d8219941d9518caa58 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 18:29:27 -0700 Subject: [PATCH 147/151] remove UpdateResources as the interface was too hard to use --- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 162 +----------------- .../gcp/healthcare/HealthcareApiClient.java | 5 + .../healthcare/HttpHealthcareApiClient.java | 15 ++ .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 29 ++++ .../sdk/io/gcp/healthcare/FhirIOUpdateIT.java | 26 +-- 5 files changed, 53 insertions(+), 184 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 0f07a8a8cc76..b4a79a36ca0e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -132,7 +132,8 @@ * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link * HealthcareIOError} containing the {@link String} that failed to be ingested and the * exception. - *

    Conditional Creating / Updating Resources

    + * + *

    Conditional Creating / Updating Resources

    * {@link FhirIO} supports interfaces for conditional update. These can be useful to handle * scenarios where an executeBundle failed. For example if you tried to create a resource that * already exists you can grab the faield bodies of your {@link FhirIO.ExecuteBundles} transform @@ -1440,165 +1441,6 @@ public void create(ProcessContext context) throws IOException { } } - /** - * Update resources update resources. - * - * @param the type parameter - * @param fhirStore the fhir store - * @return the update resources - */ - public static UpdateResources update(ValueProvider fhirStore) { - return new UpdateResources(fhirStore); - } - - /** - * Update resources update resources. - * - * @param the type parameter - * @param fhirStore the fhir store - * @return the update resources - */ - public static UpdateResources update(String fhirStore) { - return new UpdateResources(fhirStore); - } - - /** - * {@link PTransform} for Updating FHIR resources resources. - * - *

    This transform assumes the input {@link PCollection} contains {@link KV} of resource name, - * value pairs and by default will call {@code .toString} to extract string values. However, the - * user can override this behavior by specifying a {@link SerializableFunction} with custom logic - * to extract the resource name and body from the {@link KV} in {@link - * UpdateResources#withResourceNameFunction(SerializableFunction)} and {@link - * UpdateResources#withFormatBodyFunction(SerializableFunction)} - * - *

    https://cloud.google.com/healthcare/docs/reference/rest/v1beta1/projects.locations.datasets.fhirStores.fhir/update - */ - public static class UpdateResources - extends PTransform, Write.Result> { - private final ValueProvider fhirStore; - private SerializableFunction formatBodyFunction; - private SerializableFunction resourceNameFunction; - private SerializableFunction etagFunction; - public static final TupleTag UPDATE_RESULTS = new TupleTag() {}; - private static final Logger LOG = LoggerFactory.getLogger(UpdateResources.class); - - /** - * Instantiates a new Update resources. - * - * @param fhirStore the fhir store - */ - UpdateResources(ValueProvider fhirStore) { - this.fhirStore = fhirStore; - } - - /** - * Instantiates a new Update resources. - * - * @param fhirStore the fhir store - */ - UpdateResources(String fhirStore) { - this.fhirStore = StaticValueProvider.of(fhirStore); - } - /** - * Add a {@link SerializableFunction} to extract a resource name from the input element. - * - * @param resourceNameFunction the resource name function - * @return the update resources - */ - public UpdateResources withResourceNameFunction( - SerializableFunction resourceNameFunction) { - this.resourceNameFunction = resourceNameFunction; - return this; - } - - /** - * With format body function update resources. - * - * @param formatBodyFunction the format body function - * @return the update resources - */ - public UpdateResources withFormatBodyFunction( - SerializableFunction formatBodyFunction) { - this.formatBodyFunction = formatBodyFunction; - return this; - } - /** - * With ETag function update resources. - * - * @param etagFunction ETag function - * @return the update resources - */ - public UpdateResources withETagFunction(SerializableFunction etagFunction) { - this.etagFunction = etagFunction; - return this; - } - - @Override - public FhirIO.Write.Result expand(PCollection input) { - checkArgument( - formatBodyFunction != null, - "FhirIO.UpdateResources should always be called with a withFormatBodyFunction."); - checkArgument( - resourceNameFunction != null, - "FhirIO.UpdateResources should always be called with a withResourceNameFunction."); - return Write.Result.in( - input.getPipeline(), - input - .apply( - ParDo.of( - new UpdateFn( - fhirStore, formatBodyFunction, resourceNameFunction, etagFunction))) - .setCoder(HealthcareIOErrorCoder.of(StringUtf8Coder.of()))); - } - - static class UpdateFn extends DoFn> { - - private Counter failedUpdates = Metrics.counter(UpdateFn.class, "failed-updates"); - private transient HealthcareApiClient client; - private final ObjectMapper mapper = new ObjectMapper(); - private final ValueProvider fhirStore; - private SerializableFunction formatBodyFunction; - private SerializableFunction resourceNameFunction; - private SerializableFunction etagFunction; - - UpdateFn( - ValueProvider fhirStore, - SerializableFunction formatBodyFunction, - SerializableFunction resourceNameFunction, - @Nullable SerializableFunction etagFunction) { - this.fhirStore = fhirStore; - this.formatBodyFunction = formatBodyFunction; - this.resourceNameFunction = resourceNameFunction; - this.etagFunction = etagFunction; - } - - @Setup - public void initClient() throws IOException { - this.client = new HttpHealthcareApiClient(); - } - - @ProcessElement - public void update(ProcessContext context) { - T input = context.element(); - String body = formatBodyFunction.apply(input); - try { - // Validate that data was set to valid JSON. - mapper.readTree(body); - String resourceName = resourceNameFunction.apply(input); - String etag = null; - if (etagFunction != null) { - etag = etagFunction.apply(input); - } - HttpBody result = client.fhirUpdate(fhirStore.get(), resourceName, body, etag); - context.output(UPDATE_RESULTS, result.getData()); - } catch (IOException | HealthcareHttpException e) { - failedUpdates.inc(); - context.output(HealthcareIOError.of(body, e)); - } - } - } - } /** * Conditional update conditional update. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index 3f553f1c7ec7..37fdac994aff 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -164,6 +164,10 @@ HttpBody fhirConditionalUpdate( HttpBody fhirUpdate(String fhirStore, String type, String resource, @Nullable String etag) throws IOException, HealthcareHttpException; + HttpBody fhirSearch( + String fhirStore, @Nullable String type, Map searchParameters + ) throws IOException, HealthcareHttpException; + /** * Read fhir resource http body. * @@ -198,4 +202,5 @@ FhirStore createFhirStore(String dataset, String name, String version, String pu Empty deleteHL7v2Store(String store) throws IOException; Empty deleteFhirStore(String store) throws IOException; + } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 40b368d2a920..699d480ebee1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -408,6 +408,7 @@ static class FhirHttpRequest { private Map parameters; enum Method { + GET, POST, PUT } @@ -469,6 +470,9 @@ public HttpBody executeFhirHttpRequest(FhirHttpRequest fhirHttpRequest) RequestBuilder requestBuilder; switch (fhirHttpRequest.method) { + case GET: + requestBuilder = RequestBuilder.get(); + break; case PUT: requestBuilder = RequestBuilder.put(); break; @@ -564,6 +568,17 @@ public HttpBody fhirUpdate( .setMethod(Method.PUT)); } + @Override + public HttpBody fhirSearch( + String fhirStore, @Nullable String type, Map searchParameters + ) throws IOException, HealthcareHttpException { + return executeFhirHttpRequest( + new FhirHttpRequest(fhirStore, null) + .setPathSuffix(type) + .setMethod(Method.GET) + .setParameters(searchParameters)); + } + /** * Wraps {@link HttpResponse} in an exception with a statusCode field for use with {@link * HealthcareIOError}. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index b96d22bef1d0..b8754f7da651 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -22,6 +22,7 @@ import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.http.javanet.NetHttpTransport; import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.services.healthcare.v1beta1.model.HttpBody; import com.google.api.services.storage.Storage; import com.google.api.services.storage.model.StorageObject; import com.google.auth.oauth2.GoogleCredentials; @@ -41,7 +42,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; class FhirIOTestUtil { public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests"; @@ -229,4 +232,30 @@ public static void tearDownTempBucket() throws IOException { } } } + + public static class ResourceIdToNameFn extends DoFn, KV> { + private transient HealthcareApiClient client; + private ExtractIDSearchParams extractIDSearchParams = new ExtractIDSearchParams(); + private final String fhirStore; + private ObjectMapper mapper; + + public ResourceIdToNameFn(String fhirStore) { + this.fhirStore = fhirStore; + } + + @Setup + void setup() throws IOException { + client = new HttpHealthcareApiClient(); + mapper = new ObjectMapper(); + } + + @ProcessElement + void mapKeys(@Element KV idResourcePair, OutputReceiver> out) + throws IOException, HealthcareHttpException { + Map searchParams = extractIDSearchParams.apply(idResourcePair.getValue()); + String searchResults = client.fhirSearch(fhirStore, null, searchParams).getData(); + + } + } + } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java index 629c75b390b4..3d8c64c30e40 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java @@ -88,6 +88,8 @@ public void setup() throws Exception { options.setFhirStore(healthcareDataset + "/fhirStores/" + fhirStoreName); HealthcareApiClient client = new HttpHealthcareApiClient(); client.createFhirStore(healthcareDataset, fhirStoreName, version); + FhirIOTestUtil.createFhirResources( + client, healthcareDataset + "/fhirStores/" + fhirStoreName, RESOURCES.get(version)); } @After @@ -102,32 +104,8 @@ public static void teardownBucket() throws IOException { FhirIOTestUtil.tearDownTempBucket(); } - @Test - public void testFhirIO_Update() throws IOException, HealthcareHttpException { - List initialResources = RESOURCES.get(version); - FhirIOTestUtil.createFhirResources( - client, healthcareDataset + "/fhirStores/" + fhirStoreName, initialResources); - // TODO write initial resources to FHIR - Result updateResult = - (Result) - pipeline - .apply("Seed Test Resources", Create.of(initialResources)) - .apply("Extract ID keys", WithKeys.of(new GetByKey("id"))) - .apply( - "Update Resources", - FhirIO.>update(options.getFhirStore()) - .withResourceNameFunction(x -> x.getKey()) - .withFormatBodyFunction(x -> ((KV) x).getValue())); - - PAssert.that(updateResult.getFailedBodies()).empty(); - - pipeline.run().waitUntilFinish(); - // TODO spot check update results - } - @Test public void testFhirIO_ConditionalUpdate() { - // TODO write initial resources to FHIR Result conditionalUpdateResult = (Result) pipeline From ee3c99dcfd33f24ae3e8ea05f275879b3cb2c832 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 18:32:46 -0700 Subject: [PATCH 148/151] spotless --- .../java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java | 4 +--- .../beam/sdk/io/gcp/healthcare/HealthcareApiClient.java | 6 ++---- .../beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java | 4 ++-- .../apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java | 3 --- .../apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java | 5 ----- 5 files changed, 5 insertions(+), 17 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index b4a79a36ca0e..070155ed2162 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -132,8 +132,7 @@ * FhirIO.Write.Result#getFailedBodies()} to retrieve a {@link PCollection} of {@link * HealthcareIOError} containing the {@link String} that failed to be ingested and the * exception. - * - *

    Conditional Creating / Updating Resources

    + *

    Conditional Creating / Updating Resources

    * {@link FhirIO} supports interfaces for conditional update. These can be useful to handle * scenarios where an executeBundle failed. For example if you tried to create a resource that * already exists you can grab the faield bodies of your {@link FhirIO.ExecuteBundles} transform @@ -1441,7 +1440,6 @@ public void create(ProcessContext context) throws IOException { } } - /** * Conditional update conditional update. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java index 37fdac994aff..990e4aa839e7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareApiClient.java @@ -164,9 +164,8 @@ HttpBody fhirConditionalUpdate( HttpBody fhirUpdate(String fhirStore, String type, String resource, @Nullable String etag) throws IOException, HealthcareHttpException; - HttpBody fhirSearch( - String fhirStore, @Nullable String type, Map searchParameters - ) throws IOException, HealthcareHttpException; + HttpBody fhirSearch(String fhirStore, @Nullable String type, Map searchParameters) + throws IOException, HealthcareHttpException; /** * Read fhir resource http body. @@ -202,5 +201,4 @@ FhirStore createFhirStore(String dataset, String name, String version, String pu Empty deleteHL7v2Store(String store) throws IOException; Empty deleteFhirStore(String store) throws IOException; - } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 699d480ebee1..801d5d60ae9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -570,8 +570,8 @@ public HttpBody fhirUpdate( @Override public HttpBody fhirSearch( - String fhirStore, @Nullable String type, Map searchParameters - ) throws IOException, HealthcareHttpException { + String fhirStore, @Nullable String type, Map searchParameters) + throws IOException, HealthcareHttpException { return executeFhirHttpRequest( new FhirHttpRequest(fhirStore, null) .setPathSuffix(type) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index b8754f7da651..831433765666 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -22,7 +22,6 @@ import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.http.javanet.NetHttpTransport; import com.google.api.client.json.jackson2.JacksonFactory; -import com.google.api.services.healthcare.v1beta1.model.HttpBody; import com.google.api.services.storage.Storage; import com.google.api.services.storage.model.StorageObject; import com.google.auth.oauth2.GoogleCredentials; @@ -254,8 +253,6 @@ void mapKeys(@Element KV idResourcePair, OutputReceiver searchParams = extractIDSearchParams.apply(idResourcePair.getValue()); String searchResults = client.fhirSearch(fhirStore, null, searchParams).getData(); - } } - } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java index 3d8c64c30e40..8e095cc0802e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOUpdateIT.java @@ -25,18 +25,13 @@ import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; -import java.util.List; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.healthcare.FhirIO.Write.Result; import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.ExtractIDSearchParams; import org.apache.beam.sdk.io.gcp.healthcare.FhirIOTestUtil.GetByKey; -import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.values.KV; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; From 29df6d3b0548209318bca0bbc38e6e09c6a25176 Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 18:38:39 -0700 Subject: [PATCH 149/151] remove unused test util DoFn --- .../sdk/io/gcp/healthcare/FhirIOTestUtil.java | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 831433765666..8b493eb21032 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -231,28 +231,4 @@ public static void tearDownTempBucket() throws IOException { } } } - - public static class ResourceIdToNameFn extends DoFn, KV> { - private transient HealthcareApiClient client; - private ExtractIDSearchParams extractIDSearchParams = new ExtractIDSearchParams(); - private final String fhirStore; - private ObjectMapper mapper; - - public ResourceIdToNameFn(String fhirStore) { - this.fhirStore = fhirStore; - } - - @Setup - void setup() throws IOException { - client = new HttpHealthcareApiClient(); - mapper = new ObjectMapper(); - } - - @ProcessElement - void mapKeys(@Element KV idResourcePair, OutputReceiver> out) - throws IOException, HealthcareHttpException { - Map searchParams = extractIDSearchParams.apply(idResourcePair.getValue()); - String searchResults = client.fhirSearch(fhirStore, null, searchParams).getData(); - } - } } From 74c3c23d609c6af30428c59d39a030bb8d0824dd Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 18:39:51 -0700 Subject: [PATCH 150/151] fix find replace typo --- .../beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java index 801d5d60ae9d..efd2df5c812f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java @@ -598,7 +598,7 @@ public static class HealthcareHttpException extends Exception { } /** - * ConditionalUpdate Exception of {@link HttpResponse}. + * Create Exception of {@link HttpResponse}. * * @param response the HTTP response * @return the healthcare http exception From 709ed00fa06744d18ab9597de10548763e32bc4a Mon Sep 17 00:00:00 2001 From: Jacob Ferriero Date: Tue, 2 Jun 2020 19:02:16 -0700 Subject: [PATCH 151/151] remove unused imports --- .../org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java index 8b493eb21032..b96d22bef1d0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java @@ -41,9 +41,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.KV; class FhirIOTestUtil { public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests";